You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2018/03/16 23:56:51 UTC

[geode] branch develop updated: GEODE-4875: make a public ClusterConfigurationService interface

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

jinmeiliao pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 6073a15  GEODE-4875: make a public ClusterConfigurationService interface
6073a15 is described below

commit 6073a15b6cc33b93ee0fb40c1aa880d2b7af8d61
Author: Sai Boorlagadda <sa...@gmail.com>
AuthorDate: Fri Mar 16 10:57:09 2018 -0700

    GEODE-4875: make a public ClusterConfigurationService interface
    
    * rename the current ClusterConfigurationService to InternalClusterConfigurationService
    * make a public interface and have the internal implementation implements it.
---
 .../distributed/ClusterConfigurationService.java   | 24 ++++++++++++++++++++++
 .../internal/AbstractDistributionConfig.java       |  3 ++-
 ...va => InternalClusterConfigurationService.java} | 24 ++++++++++++----------
 .../distributed/internal/InternalLocator.java      | 10 ++++-----
 .../geode/distributed/internal/ServerLocator.java  |  2 +-
 .../membership/gms/locator/GMSLocator.java         |  4 ++--
 .../distributed/internal/tcpserver/TcpHandler.java |  4 ++--
 .../distributed/internal/tcpserver/TcpServer.java  |  4 ++--
 .../internal/cache/ClusterConfigurationLoader.java |  6 +++---
 .../geode/internal/cache/GemFireCacheImpl.java     |  6 +++---
 .../management/internal/JmxManagerLocator.java     |  4 ++--
 .../cli/commands/AlterAsyncEventQueueCommand.java  |  4 ++--
 .../cli/commands/CreateJndiBindingCommand.java     |  4 ++--
 .../internal/cli/commands/DeployCommand.java       |  4 ++--
 .../cli/commands/DestroyJndiBindingCommand.java    | 12 +++--------
 .../ExportImportClusterConfigurationCommands.java  |  6 +++---
 .../internal/cli/commands/GfshCommand.java         |  6 +++---
 .../cli/commands/ListJndiBindingCommand.java       |  4 ++--
 .../management/internal/cli/i18n/CliStrings.java   |  4 ++--
 .../callbacks/ConfigurationChangeListener.java     |  6 +++---
 .../functions/DownloadJarFunction.java             |  4 ++--
 .../functions/GetClusterConfigurationFunction.java |  4 ++--
 .../SharedConfigurationStatusRequestHandler.java   |  4 ++--
 .../AutoConnectionSourceImplJUnitTest.java         | 11 ++++++----
 .../LocatorLauncherIntegrationTestCase.java        |  2 +-
 .../internal/CacheConfigIntegrationTest.java       |  2 +-
 .../InternalClusterConfigurationServiceTest.java   |  4 ++--
 .../internal/tcpserver/TcpServerJUnitTest.java     |  6 +++---
 .../extension/mock/MockExtensionCommands.java      |  5 +++--
 .../commands/AlterAsyncEventQueueCommandTest.java  |  6 +++---
 .../cli/commands/AlterCompressorDUnitTest.java     |  6 +++---
 .../CreateAlterDestroyRegionCommandsDUnitTest.java |  6 +++---
 .../CreateAsyncEventQueueCommandDUnitTest.java     |  6 +++---
 .../commands/CreateAsyncEventQueueCommandTest.java |  6 +++---
 .../CreateDefinedIndexesCommandDUnitTest.java      |  6 +++---
 .../commands/CreateDefinedIndexesCommandTest.java  | 11 ++++++----
 .../commands/CreateGatewayReceiverCommandTest.java |  6 +++---
 .../commands/CreateGatewaySenderCommandTest.java   |  6 +++---
 .../CreateJndiBindingCommandDUnitTest.java         |  6 ++----
 .../cli/commands/CreateJndiBindingCommandTest.java | 20 +++++++++++-------
 .../DestroyAsyncEventQueueCommandDUnitTest.java    | 14 ++++++-------
 .../DestroyAsyncEventQueueCommandTest.java         |  4 ++--
 .../commands/DestroyGatewaySenderCommandTest.java  |  6 +++---
 .../DestroyJndiBindingCommandDUnitTest.java        |  4 ++--
 .../commands/DestroyJndiBindingCommandTest.java    |  6 +++---
 .../commands/DestroyRegionCommandDUnitTest.java    | 10 ++++-----
 .../cli/commands/DestroyRegionCommandTest.java     |  6 +++---
 .../cli/commands/DiskStoreCommandsDUnitTest.java   |  4 ++--
 .../cli/commands/GfshCommandJUnitTest.java         |  6 +++---
 .../IndexCommandsShareConfigurationDUnitTest.java  |  6 +++---
 .../internal/configuration/ClusterConfig.java      |  4 ++--
 .../ClusterConfigWithSecurityDUnitTest.java        |  6 +++---
 ...usterConfigurationServiceUsingDirDUnitTest.java |  4 ++--
 .../SharedConfigurationTestUtils.java              |  4 ++--
 .../utils/XmlUtilsAddNewNodeJUnitTest.java         |  5 +++--
 .../dunit/internal/JUnit4DistributedTestCase.java  |  4 +---
 .../LuceneClusterConfigurationDUnitTest.java       |  7 ++++---
 .../cli/commands/AlterRuntimeCommandDUnitTest.java |  4 ++--
 58 files changed, 202 insertions(+), 170 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java
new file mode 100644
index 0000000..b5a1352
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/ClusterConfigurationService.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.geode.distributed;
+
+import org.apache.geode.annotations.Experimental;
+
+@Experimental
+public interface ClusterConfigurationService {
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
index cff12fa..f4f96f8 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
@@ -1086,7 +1086,8 @@ public abstract class AbstractDistributionConfig extends AbstractConfig
         LocalizedStrings.AbstractDistributionConfig_USE_SHARED_CONFIGURATION.toLocalizedString());
     m.put(LOAD_CLUSTER_CONFIGURATION_FROM_DIR,
         LocalizedStrings.AbstractDistributionConfig_LOAD_SHARED_CONFIGURATION_FROM_DIR
-            .toLocalizedString(ClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME));
+            .toLocalizedString(
+                InternalClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME));
     m.put(CLUSTER_CONFIGURATION_DIR,
         LocalizedStrings.AbstractDistributionConfig_CLUSTER_CONFIGURATION_DIR.toLocalizedString());
     m.put(SSL_SERVER_ALIAS, LocalizedStrings.AbstractDistributionConfig_SERVER_SSL_ALIAS_0
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
similarity index 97%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
index fe65e54..73e3b9f 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalClusterConfigurationService.java
@@ -79,6 +79,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.ClusterConfigurationService;
 import org.apache.geode.distributed.DistributedLockService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
@@ -105,7 +106,7 @@ import org.apache.geode.management.internal.configuration.messages.SharedConfigu
 import org.apache.geode.management.internal.configuration.utils.XmlUtils;
 
 @SuppressWarnings({"deprecation", "unchecked"})
-public class ClusterConfigurationService {
+public class InternalClusterConfigurationService implements ClusterConfigurationService {
   private static final Logger logger = LogService.getLogger();
 
   /**
@@ -144,7 +145,7 @@ public class ClusterConfigurationService {
   private DistributedLockService sharedConfigLockingService;
 
   @TestingOnly
-  ClusterConfigurationService() {
+  InternalClusterConfigurationService() {
     configDirPath = null;
     configDiskDirPath = null;
     cache = null;
@@ -152,7 +153,7 @@ public class ClusterConfigurationService {
     initJaxbContext();
   }
 
-  public ClusterConfigurationService(InternalCache cache) throws IOException {
+  public InternalClusterConfigurationService(InternalCache cache) throws IOException {
     this.cache = cache;
     Properties properties = cache.getDistributedSystem().getProperties();
     // resolve the cluster config dir
@@ -230,7 +231,7 @@ public class ClusterConfigurationService {
     try {
       Region<String, Configuration> configRegion = getConfigurationRegion();
       if (groups == null || groups.length == 0) {
-        groups = new String[] {ClusterConfigurationService.CLUSTER_CONFIG};
+        groups = new String[] {InternalClusterConfigurationService.CLUSTER_CONFIG};
       }
       for (String group : groups) {
         Configuration configuration = configRegion.get(group);
@@ -299,7 +300,7 @@ public class ClusterConfigurationService {
     lockSharedConfiguration();
     try {
       if (groups == null) {
-        groups = new String[] {ClusterConfigurationService.CLUSTER_CONFIG};
+        groups = new String[] {InternalClusterConfigurationService.CLUSTER_CONFIG};
       }
       Region<String, Configuration> configRegion = getConfigurationRegion();
       for (String group : groups) {
@@ -345,7 +346,7 @@ public class ClusterConfigurationService {
     lockSharedConfiguration();
     try {
       if (groups == null) {
-        groups = new String[] {ClusterConfigurationService.CLUSTER_CONFIG};
+        groups = new String[] {InternalClusterConfigurationService.CLUSTER_CONFIG};
       }
       Region<String, Configuration> configRegion = getConfigurationRegion();
       for (String group : groups) {
@@ -507,7 +508,7 @@ public class ClusterConfigurationService {
     try {
       if (loadSharedConfigFromDir) {
         logger.info("Reading cluster configuration from '{}' directory",
-            ClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME);
+            InternalClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME);
         loadSharedConfigurationFromDisk();
       } else {
         persistSecuritySettings(configRegion);
@@ -534,10 +535,11 @@ public class ClusterConfigurationService {
     Properties securityProps = this.cache.getDistributedSystem().getSecurityProperties();
 
     Configuration clusterPropertiesConfig =
-        configRegion.get(ClusterConfigurationService.CLUSTER_CONFIG);
+        configRegion.get(InternalClusterConfigurationService.CLUSTER_CONFIG);
     if (clusterPropertiesConfig == null) {
-      clusterPropertiesConfig = new Configuration(ClusterConfigurationService.CLUSTER_CONFIG);
-      configRegion.put(ClusterConfigurationService.CLUSTER_CONFIG, clusterPropertiesConfig);
+      clusterPropertiesConfig =
+          new Configuration(InternalClusterConfigurationService.CLUSTER_CONFIG);
+      configRegion.put(InternalClusterConfigurationService.CLUSTER_CONFIG, clusterPropertiesConfig);
     }
     // put security-manager and security-post-processor in the cluster config
     Properties clusterProperties = clusterPropertiesConfig.getGemfireProperties();
@@ -562,7 +564,7 @@ public class ClusterConfigurationService {
     try {
       if (isLocked) {
         configResponse = new ConfigurationResponse();
-        groups.add(ClusterConfigurationService.CLUSTER_CONFIG);
+        groups.add(InternalClusterConfigurationService.CLUSTER_CONFIG);
         logger.info("Building up configuration response with following configurations: {}", groups);
 
         for (String group : groups) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 955723d..0535277 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -173,7 +173,7 @@ public class InternalLocator extends Locator implements ConnectListener {
 
   private final AtomicBoolean shutdownHandled = new AtomicBoolean(false);
 
-  private ClusterConfigurationService sharedConfig;
+  private InternalClusterConfigurationService sharedConfig;
 
   private volatile boolean isSharedConfigurationStarted = false;
 
@@ -485,7 +485,7 @@ public class InternalLocator extends Locator implements ConnectListener {
     this.server.start();
   }
 
-  public ClusterConfigurationService getSharedConfiguration() {
+  public InternalClusterConfigurationService getSharedConfiguration() {
     return this.sharedConfig;
   }
 
@@ -1042,7 +1042,7 @@ public class InternalLocator extends Locator implements ConnectListener {
       }
       this.productUseLog.monitorUse(newSystem);
       if (isSharedConfigurationEnabled()) {
-        this.sharedConfig = new ClusterConfigurationService(newCache);
+        this.sharedConfig = new InternalClusterConfigurationService(newCache);
         startSharedConfigurationService();
       }
       if (!this.server.isAlive()) {
@@ -1163,7 +1163,7 @@ public class InternalLocator extends Locator implements ConnectListener {
 
     @Override
     public void restarting(DistributedSystem ds, GemFireCache cache,
-        ClusterConfigurationService sharedConfig) {
+        InternalClusterConfigurationService sharedConfig) {
       if (ds != null) {
         for (TcpHandler handler : this.allHandlers) {
           handler.restarting(ds, cache, sharedConfig);
@@ -1327,7 +1327,7 @@ public class InternalLocator extends Locator implements ConnectListener {
     try {
       if (locator.sharedConfig == null) {
         // locator.sharedConfig will already be created in case of auto-reconnect
-        locator.sharedConfig = new ClusterConfigurationService(locator.myCache);
+        locator.sharedConfig = new InternalClusterConfigurationService(locator.myCache);
       }
       locator.sharedConfig.initSharedConfiguration(locator.loadFromSharedConfigDir());
       logger.info(
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
index a21bb3b..9ec8a00 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
@@ -279,7 +279,7 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
   }
 
   public void restarting(DistributedSystem ds, GemFireCache cache,
-      ClusterConfigurationService sharedConfig) {
+      InternalClusterConfigurationService sharedConfig) {
     if (ds != null) {
       this.loadSnapshot = new LocatorLoadSnapshot();
       this.ds = (InternalDistributedSystem) ds;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
index 922d8f9..5062fbf 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
@@ -40,8 +40,8 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.LocatorStats;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -385,7 +385,7 @@ public class GMSLocator implements Locator, NetLocator {
 
   @Override
   public void restarting(DistributedSystem ds, GemFireCache cache,
-      ClusterConfigurationService sharedConfig) {
+      InternalClusterConfigurationService sharedConfig) {
     setMembershipManager(((InternalDistributedSystem) ds).getDM().getMembershipManager());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpHandler.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpHandler.java
index d18b7cb..a086901 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpHandler.java
@@ -18,7 +18,7 @@ import java.io.IOException;
 
 import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 
 /**
  * A handler which responds to messages for the {@link TcpServer}
@@ -50,7 +50,7 @@ public interface TcpHandler {
    * @param sharedConfig TODO
    */
   void restarting(DistributedSystem ds, GemFireCache cache,
-      ClusterConfigurationService sharedConfig);
+      InternalClusterConfigurationService sharedConfig);
 
   /**
    * Initialize the handler with the TcpServer. Called before the TcpServer starts accepting
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
index 008156c..169b09d 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
@@ -44,10 +44,10 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.IncompatibleVersionException;
 import org.apache.geode.cache.UnsupportedVersionException;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.PoolStatHelper;
@@ -204,7 +204,7 @@ public class TcpServer {
   }
 
   public void restarting(InternalDistributedSystem ds, InternalCache cache,
-      ClusterConfigurationService sharedConfig) throws IOException {
+      InternalClusterConfigurationService sharedConfig) throws IOException {
     this.shuttingDown = false;
     this.handler.restarting(ds, cache, sharedConfig);
     startServerThread();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
index 95520ad..7cc56cd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
@@ -51,8 +51,8 @@ import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.ConfigSource;
@@ -174,7 +174,7 @@ public class ClusterConfigurationLoader {
 
     // apply the cluster config first
     Configuration clusterConfiguration =
-        requestedConfiguration.get(ClusterConfigurationService.CLUSTER_CONFIG);
+        requestedConfiguration.get(InternalClusterConfigurationService.CLUSTER_CONFIG);
     if (clusterConfiguration != null) {
       String cacheXmlContent = clusterConfiguration.getCacheXmlContent();
       if (StringUtils.isNotBlank(cacheXmlContent)) {
@@ -226,7 +226,7 @@ public class ClusterConfigurationLoader {
 
     // apply the cluster config first
     Configuration clusterConfiguration =
-        requestedConfiguration.get(ClusterConfigurationService.CLUSTER_CONFIG);
+        requestedConfiguration.get(InternalClusterConfigurationService.CLUSTER_CONFIG);
     if (clusterConfiguration != null) {
       runtimeProps.putAll(clusterConfiguration.getGemfireProperties());
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 5fd2b96..a82e152 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -154,13 +154,13 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.CacheTime;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionAdvisee;
 import org.apache.geode.distributed.internal.DistributionAdvisor;
 import org.apache.geode.distributed.internal.DistributionAdvisor.Profile;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.PooledExecutorWithDMStats;
@@ -1056,8 +1056,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
           .create(LocalizedStrings.GemFireCache_RECEIVED_SHARED_CONFIGURATION_FROM_LOCATORS));
       logger.info(response.describeConfig());
 
-      Configuration clusterConfig =
-          response.getRequestedConfiguration().get(ClusterConfigurationService.CLUSTER_CONFIG);
+      Configuration clusterConfig = response.getRequestedConfiguration()
+          .get(InternalClusterConfigurationService.CLUSTER_CONFIG);
       Properties clusterSecProperties =
           clusterConfig == null ? new Properties() : clusterConfig.getGemfireProperties();
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocator.java b/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocator.java
index cc7036b..fc72de7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocator.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
 import org.apache.geode.distributed.internal.tcpserver.TcpServer;
@@ -70,7 +70,7 @@ public class JmxManagerLocator implements TcpHandler {
 
   @Override
   public void restarting(DistributedSystem ds, GemFireCache cache,
-      ClusterConfigurationService sharedConfig) {
+      InternalClusterConfigurationService sharedConfig) {
     this.cache = (InternalCache) cache;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
index 64fc940..c1f5a5a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommand.java
@@ -34,7 +34,7 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.AbstractCliAroundInterceptor;
@@ -85,7 +85,7 @@ public class AlterAsyncEventQueueCommand extends GfshCommand {
 
     // need not check if any running servers has this async-event-queue. A server with this queue id
     // may be shutdown, but we still need to update Cluster Configuration.
-    ClusterConfigurationService service = getSharedConfiguration();
+    InternalClusterConfigurationService service = getSharedConfiguration();
 
     if (service == null) {
       return ResultBuilder.createUserErrorResult("Cluster Configuration Service is not available. "
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
index 9d6d7da..6c0e04c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
@@ -26,7 +26,7 @@ import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.configuration.CacheConfig;
 import org.apache.geode.internal.cache.configuration.JndiBindingsType;
 import org.apache.geode.internal.logging.LogService;
@@ -147,7 +147,7 @@ public class CreateJndiBindingCommand extends GfshCommand {
 
     Result result;
     boolean persisted = false;
-    ClusterConfigurationService service = getSharedConfiguration();
+    InternalClusterConfigurationService service = getSharedConfiguration();
 
     if (service != null) {
       CacheConfig cacheConfig = service.getCacheConfig("cluster");
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
index 23cf600..f71fa64 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
@@ -36,7 +36,7 @@ import org.springframework.shell.core.annotation.CliOption;
 
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
@@ -136,7 +136,7 @@ public class DeployCommand extends GfshCommand {
     }
 
     Result result = ResultBuilder.buildResult(tabularData);
-    ClusterConfigurationService sc = getSharedConfiguration();
+    InternalClusterConfigurationService sc = getSharedConfiguration();
     if (sc == null) {
       result.setCommandPersisted(false);
     } else {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
index dd62e58..3913381 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
@@ -16,19 +16,14 @@ package org.apache.geode.management.internal.cli.commands;
 
 import static org.apache.geode.management.internal.cli.result.ResultBuilder.buildResult;
 
-import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.TransformerException;
-
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
-import org.xml.sax.SAXException;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.configuration.JndiBindingsType;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result;
@@ -56,12 +51,11 @@ public class DestroyJndiBindingCommand extends GfshCommand {
   public Result destroyJDNIBinding(
       @CliOption(key = JNDI_NAME, mandatory = true, help = JNDI_NAME__HELP) String jndiName,
       @CliOption(key = CliStrings.IFEXISTS, help = IFEXISTS_HELP, specifiedDefaultValue = "true",
-          unspecifiedDefaultValue = "false") boolean ifExists)
-      throws IOException, SAXException, ParserConfigurationException, TransformerException {
+          unspecifiedDefaultValue = "false") boolean ifExists) {
 
     Result result;
     boolean persisted = false;
-    ClusterConfigurationService service = getSharedConfiguration();
+    InternalClusterConfigurationService service = getSharedConfiguration();
     if (service != null) {
       service.updateCacheConfig("cluster", cc -> {
         List<JndiBindingsType.JndiBinding> bindings = cc.getJndiBindings();
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
index 319b5c0..9179c5c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
@@ -37,7 +37,7 @@ import org.xml.sax.SAXException;
 
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.cli.CliMetaData;
@@ -93,7 +93,7 @@ public class ExportImportClusterConfigurationCommands extends GfshCommand {
     }
 
     File zipFile = tempDir.resolve("exportedCC.zip").toFile();
-    ClusterConfigurationService sc = locator.getSharedConfiguration();
+    InternalClusterConfigurationService sc = locator.getSharedConfiguration();
 
     Result result;
     try {
@@ -158,7 +158,7 @@ public class ExportImportClusterConfigurationCommands extends GfshCommand {
     InfoResultData infoData = ResultBuilder.createInfoResultData();
     String zipFilePath = filePathFromShell.get(0);
 
-    ClusterConfigurationService sc = locator.getSharedConfiguration();
+    InternalClusterConfigurationService sc = locator.getSharedConfiguration();
 
     // backup the old config
     for (Configuration config : sc.getConfigurationRegion().values()) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
index a4b4ef8..2beeb7a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.security.SecurityService;
@@ -65,7 +65,7 @@ public abstract class GfshCommand implements CommandMarker {
     return getGfsh() != null && getGfsh().isConnectedAndReady();
   }
 
-  public ClusterConfigurationService getSharedConfiguration() {
+  public InternalClusterConfigurationService getSharedConfiguration() {
     InternalLocator locator = InternalLocator.getLocator();
     return locator == null ? null : locator.getSharedConfiguration();
   }
@@ -74,7 +74,7 @@ public abstract class GfshCommand implements CommandMarker {
     if (result == null) {
       throw new IllegalArgumentException("Result should not be null");
     }
-    ClusterConfigurationService sc = getSharedConfiguration();
+    InternalClusterConfigurationService sc = getSharedConfiguration();
     if (sc == null) {
       result.setCommandPersisted(false);
     } else {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListJndiBindingCommand.java
index b2658ae..b895ead 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListJndiBindingCommand.java
@@ -24,7 +24,7 @@ import org.springframework.shell.core.annotation.CliCommand;
 
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.configuration.CacheConfig;
 import org.apache.geode.internal.cache.configuration.JndiBindingsType;
 import org.apache.geode.internal.logging.LogService;
@@ -54,7 +54,7 @@ public class ListJndiBindingCommand extends GfshCommand {
     TabularResultData configTable = null;
     TabularResultData memberTable = null;
 
-    ClusterConfigurationService ccService = getSharedConfiguration();
+    InternalClusterConfigurationService ccService = getSharedConfiguration();
     if (ccService != null) {
       configTable = resultSection.addTable();
       configTable.setHeader("Configured JNDI bindings: ");
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index fe8650c..ba4138d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -41,8 +41,8 @@ import java.text.MessageFormat;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 
@@ -2410,7 +2410,7 @@ public class CliStrings {
   public static final String START_LOCATOR__LOAD__SHARED_CONFIGURATION__FROM__FILESYSTEM__HELP =
       "When \" " + START_LOCATOR__LOAD__SHARED_CONFIGURATION__FROM__FILESYSTEM
           + " \" is set to true, the locator loads the cluster configuration from the \""
-          + ClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME + "\" directory.";
+          + InternalClusterConfigurationService.CLUSTER_CONFIG_ARTIFACTS_DIR_NAME + "\" directory.";
   public static final String START_LOCATOR__CLUSTER__CONFIG__DIR = "cluster-config-dir";
   public static final String START_LOCATOR__CLUSTER__CONFIG__DIR__HELP =
       "Directory used by the cluster configuration service to store the cluster configuration on the filesystem";
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/callbacks/ConfigurationChangeListener.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/callbacks/ConfigurationChangeListener.java
index d1945c2..7e57b47 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/callbacks/ConfigurationChangeListener.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/callbacks/ConfigurationChangeListener.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
@@ -39,9 +39,9 @@ import org.apache.geode.management.internal.configuration.domain.Configuration;
 public class ConfigurationChangeListener extends CacheListenerAdapter<String, Configuration> {
   private static final Logger logger = LogService.getLogger();
 
-  private final ClusterConfigurationService sharedConfig;
+  private final InternalClusterConfigurationService sharedConfig;
 
-  public ConfigurationChangeListener(ClusterConfigurationService sharedConfig) {
+  public ConfigurationChangeListener(InternalClusterConfigurationService sharedConfig) {
     this.sharedConfig = sharedConfig;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/DownloadJarFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/DownloadJarFunction.java
index 345f144..0fb7517 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/DownloadJarFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/DownloadJarFunction.java
@@ -30,7 +30,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.execute.InternalFunction;
 import org.apache.geode.internal.logging.LogService;
@@ -50,7 +50,7 @@ public class DownloadJarFunction implements InternalFunction<Object[]> {
 
     RemoteInputStream result = null;
     if (locator != null && group != null && jarName != null) {
-      ClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
+      InternalClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
       if (sharedConfig != null) {
         try {
           File jarFile = sharedConfig.getPathToJarOnThisLocator(group, jarName).toFile();
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/GetClusterConfigurationFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/GetClusterConfigurationFunction.java
index 0f9addc..5b70150 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/GetClusterConfigurationFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/functions/GetClusterConfigurationFunction.java
@@ -21,7 +21,7 @@ import java.util.Set;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.execute.InternalFunction;
 import org.apache.geode.internal.logging.LogService;
@@ -32,7 +32,7 @@ public class GetClusterConfigurationFunction implements InternalFunction {
 
   @Override
   public void execute(FunctionContext context) {
-    ClusterConfigurationService clusterConfigurationService =
+    InternalClusterConfigurationService clusterConfigurationService =
         InternalLocator.getLocator().getSharedConfiguration();
 
     Set<String> groups = (Set<String>) context.getArguments();
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/handlers/SharedConfigurationStatusRequestHandler.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/handlers/SharedConfigurationStatusRequestHandler.java
index a87e079..1a7db96 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/handlers/SharedConfigurationStatusRequestHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/handlers/SharedConfigurationStatusRequestHandler.java
@@ -18,7 +18,7 @@ import java.io.IOException;
 
 import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
 import org.apache.geode.distributed.internal.tcpserver.TcpServer;
@@ -56,7 +56,7 @@ public class SharedConfigurationStatusRequestHandler implements TcpHandler {
 
   @Override
   public void restarting(DistributedSystem system, GemFireCache cache,
-      ClusterConfigurationService sharedConfig) {
+      InternalClusterConfigurationService sharedConfig) {
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
index 1a213ee..e199654 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.cache.client.internal;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -47,7 +46,11 @@ import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.*;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.GemFireCache;
+import org.apache.geode.cache.NoSubscriptionServersAvailableException;
+import org.apache.geode.cache.RegionService;
 import org.apache.geode.cache.client.NoAvailableLocatorsException;
 import org.apache.geode.cache.client.SubscriptionNotEnabledException;
 import org.apache.geode.cache.client.internal.locator.ClientConnectionRequest;
@@ -55,8 +58,8 @@ import org.apache.geode.cache.client.internal.locator.ClientConnectionResponse;
 import org.apache.geode.cache.client.internal.locator.LocatorListResponse;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.distributed.internal.ServerLocation;
@@ -416,7 +419,7 @@ public class AutoConnectionSourceImplJUnitTest {
     public void endResponse(Object request, long startTime) {}
 
     public void restarting(DistributedSystem ds, GemFireCache cache,
-        ClusterConfigurationService sharedConfig) {}
+        InternalClusterConfigurationService sharedConfig) {}
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
index c1d2505..d7c7e9c 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
@@ -20,8 +20,8 @@ import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_CONFI
 import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_AUTO_RECONNECT;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.internal.ClusterConfigurationService.CLUSTER_CONFIG_DISK_DIR_PREFIX;
 import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+import static org.apache.geode.distributed.internal.InternalClusterConfigurationService.CLUSTER_CONFIG_DISK_DIR_PREFIX;
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.internal.DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY;
 import static org.assertj.core.api.Assertions.assertThat;
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
index 3e13846..e8cf33b 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/CacheConfigIntegrationTest.java
@@ -44,7 +44,7 @@ public class CacheConfigIntegrationTest {
     xmlFile = temporaryFolder.newFile("cache.xml");
     CacheConfig cacheConfig = new CacheConfig();
     cacheConfig.setVersion("1.0");
-    ClusterConfigurationService service = new ClusterConfigurationService();
+    InternalClusterConfigurationService service = new InternalClusterConfigurationService();
     String xml = service.marshall(cacheConfig);
     FileUtils.writeStringToFile(xmlFile, xml, "UTF-8");
 
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
index df8cf9b..1cf6062 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalClusterConfigurationServiceTest.java
@@ -46,12 +46,12 @@ import org.apache.geode.test.junit.categories.UnitTest;
 public class InternalClusterConfigurationServiceTest {
   private String xml;
   private CacheConfig unmarshalled;
-  private ClusterConfigurationService service;
+  private InternalClusterConfigurationService service;
   private Configuration configuration;
 
   @Before
   public void setUp() throws Exception {
-    service = spy(ClusterConfigurationService.class);
+    service = spy(InternalClusterConfigurationService.class);
     configuration = new Configuration("cluster");
     doReturn(configuration).when(service).getConfiguration(any());
     doReturn(mock(Region.class)).when(service).getConfigurationRegion();
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
index 0c46e4f..de84830 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
@@ -38,8 +38,8 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.DataSerializable;
 import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.tier.sockets.TcpServerFactory;
@@ -183,7 +183,7 @@ public class TcpServerJUnitTest {
     }
 
     public void restarting(DistributedSystem ds, GemFireCache cache,
-        ClusterConfigurationService sharedConfig) {}
+        InternalClusterConfigurationService sharedConfig) {}
 
     public void endRequest(Object request, long startTime) {}
 
@@ -216,7 +216,7 @@ public class TcpServerJUnitTest {
     public void shutDown() {}
 
     public void restarting(DistributedSystem ds, GemFireCache cache,
-        ClusterConfigurationService sharedConfig) {}
+        InternalClusterConfigurationService sharedConfig) {}
 
     public void endRequest(Object request, long startTime) {}
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/extension/mock/MockExtensionCommands.java b/geode-core/src/test/java/org/apache/geode/internal/cache/extension/mock/MockExtensionCommands.java
index 1322e4a..cfd1e20 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/extension/mock/MockExtensionCommands.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/extension/mock/MockExtensionCommands.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
@@ -202,7 +202,8 @@ public class MockExtensionCommands implements CommandMarker {
 
     final Result result = ResultBuilder.buildResult(tabularResultData);
 
-    ClusterConfigurationService ccService = InternalLocator.getLocator().getSharedConfiguration();
+    InternalClusterConfigurationService ccService =
+        InternalLocator.getLocator().getSharedConfiguration();
     System.out.println("MockExtensionCommands: persisting xmlEntity=" + xmlEntity);
     if (null != xmlEntity.get()) {
       if (addXmlElement) {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
index c9d5104..d5658e7 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
@@ -40,7 +40,7 @@ import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.utils.XmlUtils;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -54,13 +54,13 @@ public class AlterAsyncEventQueueCommandTest {
   public static GfshParserRule gfsh = new GfshParserRule();
 
   private AlterAsyncEventQueueCommand command;
-  private ClusterConfigurationService service;
+  private InternalClusterConfigurationService service;
   private Region<String, Configuration> configRegion;
 
   @Before
   public void before() throws Exception {
     command = spy(AlterAsyncEventQueueCommand.class);
-    service = mock(ClusterConfigurationService.class);
+    service = mock(InternalClusterConfigurationService.class);
     doReturn(service).when(command).getSharedConfiguration();
     configRegion = mock(Region.class);
     when(service.getConfigurationRegion()).thenReturn(configRegion);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
index b91ce64..a7eb966 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
@@ -29,7 +29,7 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.CachedDeserializable;
 import org.apache.geode.internal.cache.CachedDeserializableFactory;
 import org.apache.geode.internal.cache.EntryEventImpl;
@@ -129,7 +129,7 @@ public class AlterCompressorDUnitTest {
       // add the compressor to the region attributes and put it back in cluster config
       // this is just a hack to change the cache.xml so that when server restarts it restarts
       // with new region attributes
-      ClusterConfigurationService ccService =
+      InternalClusterConfigurationService ccService =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration configuration = ccService.getConfiguration("dataStore");
       String modifiedXml =
@@ -170,7 +170,7 @@ public class AlterCompressorDUnitTest {
       // remove the compressor to the region attributes and put it back in cluster config
       // this is just a hack to change the cache.xml so that when server restarts it restarts
       // with new region attributes
-      ClusterConfigurationService ccService =
+      InternalClusterConfigurationService ccService =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration configuration = ccService.getConfiguration("dataStore");
       String modifiedXml = removeCompressor(configuration.getCacheXmlContent());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
index eaf714c..596740d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
@@ -55,7 +55,7 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.AvailablePortHelper;
@@ -180,7 +180,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
     // Make sure the region exists in the shared config
     Host.getHost(0).getVM(0).invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       try {
         assertTrue(
@@ -222,7 +222,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
     // Make sure the region was altered in the shared config
     Host.getHost(0).getVM(0).invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       try {
         assertTrue(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains("45635"));
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
index 0075e90..01ad399 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
@@ -21,7 +21,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.wan.MyAsyncEventListener;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.test.dunit.IgnoredException;
@@ -110,7 +110,7 @@ public class CreateAsyncEventQueueCommandDUnitTest {
     gfsh.connectAndVerify(locator);
 
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       assertThat(service.getConfiguration("cluster").getCacheXmlContent())
           .doesNotContain("async-event-queue");
@@ -120,7 +120,7 @@ public class CreateAsyncEventQueueCommandDUnitTest {
         .tableHasRowCount("Member", 1).tableHasColumnWithExactValuesInAnyOrder("Status", "Success");
 
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration configuration = service.getConfiguration("cluster");
       configuration.getCacheXmlContent().contains("id=queue");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandTest.java
index 4c4f47d..206a2c7 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandTest.java
@@ -45,7 +45,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.execute.Function;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
@@ -62,12 +62,12 @@ public class CreateAsyncEventQueueCommandTest {
   public static GfshParserRule gfsh = new GfshParserRule();
 
   private CreateAsyncEventQueueCommand command;
-  private ClusterConfigurationService service;
+  private InternalClusterConfigurationService service;
 
   @Before
   public void before() throws Exception {
     command = spy(CreateAsyncEventQueueCommand.class);
-    service = mock(ClusterConfigurationService.class);
+    service = mock(InternalClusterConfigurationService.class);
     doReturn(service).when(command).getSharedConfiguration();
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
index 862e745..3f3f24a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
@@ -139,7 +139,7 @@ public class CreateDefinedIndexesCommandDUnitTest {
 
     locator.invoke(() -> {
       // Make sure the indexes exist in the cluster config
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       assertThat(sharedConfig.getConfiguration("cluster").getCacheXmlContent()).contains(index1Name,
           index2Name);
@@ -201,7 +201,7 @@ public class CreateDefinedIndexesCommandDUnitTest {
 
     locator.invoke(() -> {
       // Make sure the indexes exist in the cluster config
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       assertThat(sharedConfig.getConfiguration("group1").getCacheXmlContent()).contains(index2Name,
           index1Name);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
index 3435d0c..99f5718 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandTest.java
@@ -40,7 +40,7 @@ import org.mockito.Mockito;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.query.IndexType;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.internal.cli.domain.IndexInfo;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.result.CommandResult;
@@ -86,7 +86,8 @@ public class CreateDefinedIndexesCommandTest {
   public void creationFailure() throws Exception {
     DistributedMember member = mock(DistributedMember.class);
     when(member.getId()).thenReturn("memberId");
-    ClusterConfigurationService mockService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService mockService =
+        mock(InternalClusterConfigurationService.class);
 
     doReturn(mockService).when(command).getSharedConfiguration();
     doReturn(Collections.singleton(member)).when(command).findMembers(any(), any());
@@ -106,7 +107,8 @@ public class CreateDefinedIndexesCommandTest {
     XmlEntity xmlEntity = mock(XmlEntity.class);
     DistributedMember member = mock(DistributedMember.class);
     when(member.getId()).thenReturn("memberId");
-    ClusterConfigurationService mockService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService mockService =
+        mock(InternalClusterConfigurationService.class);
 
     doReturn(mockService).when(command).getSharedConfiguration();
     doReturn(Collections.singleton(member)).when(command).findMembers(any(), any());
@@ -132,7 +134,8 @@ public class CreateDefinedIndexesCommandTest {
     when(member1.getId()).thenReturn("memberId_1");
     when(member2.getId()).thenReturn("memberId_2");
 
-    ClusterConfigurationService mockService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService mockService =
+        mock(InternalClusterConfigurationService.class);
     CliFunctionResult member1Region1Result =
         new CliFunctionResult(member1.getId(), xmlEntityRegion1);
     CliFunctionResult member1Region2Result =
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommandTest.java
index 161c6ef..384bcbb 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewayReceiverCommandTest.java
@@ -32,7 +32,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
@@ -50,14 +50,14 @@ public class CreateGatewayReceiverCommandTest {
   private CreateGatewayReceiverCommand command;
   private InternalCache cache;
   private List<CliFunctionResult> functionResults;
-  private ClusterConfigurationService ccService;
+  private InternalClusterConfigurationService ccService;
   private CliFunctionResult result1;
   private XmlEntity xmlEntity;
 
   @Before
   public void before() {
     command = spy(CreateGatewayReceiverCommand.class);
-    ccService = mock(ClusterConfigurationService.class);
+    ccService = mock(InternalClusterConfigurationService.class);
     xmlEntity = mock(XmlEntity.class);
     cache = mock(InternalCache.class);
     doReturn(cache).when(command).getCache();
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommandTest.java
index f8a0ae3..122cf10 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateGatewaySenderCommandTest.java
@@ -33,7 +33,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
@@ -49,14 +49,14 @@ public class CreateGatewaySenderCommandTest {
   private CreateGatewaySenderCommand command;
   private InternalCache cache;
   private List<CliFunctionResult> functionResults;
-  private ClusterConfigurationService ccService;
+  private InternalClusterConfigurationService ccService;
   private CliFunctionResult result1, result2;
   private XmlEntity xmlEntity;
 
   @Before
   public void before() {
     command = spy(CreateGatewaySenderCommand.class);
-    ccService = mock(ClusterConfigurationService.class);
+    ccService = mock(InternalClusterConfigurationService.class);
     xmlEntity = mock(XmlEntity.class);
     cache = mock(InternalCache.class);
     doReturn(cache).when(command).getCache();
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandDUnitTest.java
index 76d293e..91f0785 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandDUnitTest.java
@@ -17,8 +17,6 @@ package org.apache.geode.management.internal.cli.commands;
 
 import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
 
-import java.util.stream.Stream;
-
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -27,7 +25,7 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.jndi.JNDIInvoker;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.utils.XmlUtils;
@@ -71,7 +69,7 @@ public class CreateJndiBindingCommandDUnitTest {
 
     // verify cluster config is updated
     locator.invoke(() -> {
-      ClusterConfigurationService ccService =
+      InternalClusterConfigurationService ccService =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration configuration = ccService.getConfiguration("cluster");
       Document document = XmlUtils.createDocumentFromXml(configuration.getCacheXmlContent());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
index 19b3854..f7fd35f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
@@ -40,7 +40,7 @@ import org.mockito.ArgumentCaptor;
 import org.xml.sax.SAXException;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.configuration.CacheConfig;
 import org.apache.geode.internal.cache.configuration.JndiBindingsType;
@@ -98,7 +98,8 @@ public class CreateJndiBindingCommandTest {
   @Test
   public void returnsErrorIfBindingAlreadyExistsAndIfUnspecified()
       throws ParserConfigurationException, SAXException, IOException {
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
     JndiBindingsType.JndiBinding existingBinding = mock(JndiBindingsType.JndiBinding.class);
 
@@ -114,7 +115,8 @@ public class CreateJndiBindingCommandTest {
   @Test
   public void skipsIfBindingAlreadyExistsAndIfSpecified()
       throws ParserConfigurationException, SAXException, IOException {
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
     JndiBindingsType.JndiBinding existingBinding = mock(JndiBindingsType.JndiBinding.class);
 
@@ -131,7 +133,8 @@ public class CreateJndiBindingCommandTest {
   @Test
   public void skipsIfBindingAlreadyExistsAndIfSpecifiedTrue()
       throws ParserConfigurationException, SAXException, IOException {
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
     JndiBindingsType.JndiBinding existingBinding = mock(JndiBindingsType.JndiBinding.class);
 
@@ -147,7 +150,8 @@ public class CreateJndiBindingCommandTest {
 
   @Test
   public void returnsErrorIfBindingAlreadyExistsAndIfSpecifiedFalse() {
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
     JndiBindingsType.JndiBinding existingBinding = mock(JndiBindingsType.JndiBinding.class);
 
@@ -174,7 +178,8 @@ public class CreateJndiBindingCommandTest {
 
   @Test
   public void whenNoMembersFoundAndClusterConfigRunningThenUpdateClusterConfig() {
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
 
     doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
@@ -238,7 +243,8 @@ public class CreateJndiBindingCommandTest {
         "Tried creating jndi binding \"name\" on \"server1\"");
     List<CliFunctionResult> results = new ArrayList<>();
     results.add(result);
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    InternalClusterConfigurationService clusterConfigService =
+        mock(InternalClusterConfigurationService.class);
     CacheConfig cacheConfig = mock(CacheConfig.class);
 
     doReturn(members).when(command).findMembers(any(), any());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandDUnitTest.java
index f621785..08680db 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandDUnitTest.java
@@ -23,7 +23,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.wan.MyAsyncEventListener;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.json.GfJsonException;
@@ -62,7 +62,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
     gfsh.executeAndAssertThat("list async-event-queues").statusIsSuccess();
 
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration config = service.getConfiguration("cluster");
       assertThat(config.getCacheXmlContent()).contains("id=\"queue1\"");
@@ -74,7 +74,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
 
     // verify that aeq entry is deleted from cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration config = service.getConfiguration("cluster");
       assertThat(config.getCacheXmlContent()).doesNotContain("id=\"queue1\"");
@@ -113,7 +113,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
 
     // verify that aeq entry is deleted from cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration config = service.getConfiguration("cluster");
       assertThat(config.getCacheXmlContent()).doesNotContain("id=\"queue1\"");
@@ -134,7 +134,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
 
     // verify that aeq entry is deleted from cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration config = service.getConfiguration("group1");
       assertThat(config.getCacheXmlContent()).doesNotContain("id=\"queue1\"");
@@ -154,7 +154,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
 
     // verify that aeq entry is not deleted from cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration config = service.getConfiguration("group1");
       assertThat(config.getCacheXmlContent()).contains("id=\"queue1\"");
@@ -204,7 +204,7 @@ public class DestroyAsyncEventQueueCommandDUnitTest {
 
     // verify that cluster config aeq entry for destroyed queue is deleted
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       System.out.println("cluster config: " + service.getConfiguration("cluster"));
       Configuration config1 = service.getConfiguration("group1");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandTest.java
index 04903a0..f1b8f8d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyAsyncEventQueueCommandTest.java
@@ -36,7 +36,7 @@ import org.mockito.ArgumentCaptor;
 
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.functions.DestroyAsyncEventQueueFunction;
@@ -51,7 +51,7 @@ public class DestroyAsyncEventQueueCommandTest {
   public static GfshParserRule gfsh = new GfshParserRule();
 
   private DestroyAsyncEventQueueCommand command;
-  private ClusterConfigurationService service;
+  private InternalClusterConfigurationService service;
   private Map<String, Configuration> configurationMap;
   private DistributedMember member1 = mock(DistributedMember.class);
   private DistributedMember member2 = mock(DistributedMember.class);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommandTest.java
index 8aa971e..1fe2b12 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyGatewaySenderCommandTest.java
@@ -32,7 +32,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
@@ -49,14 +49,14 @@ public class DestroyGatewaySenderCommandTest {
   private DestroyGatewaySenderCommand command;
   private InternalCache cache;
   private List<CliFunctionResult> functionResults;
-  private ClusterConfigurationService ccService;
+  private InternalClusterConfigurationService ccService;
   private CliFunctionResult result1, result2;
   private XmlEntity xmlEntity;
 
   @Before
   public void before() throws Exception {
     command = spy(DestroyGatewaySenderCommand.class);
-    ccService = mock(ClusterConfigurationService.class);
+    ccService = mock(InternalClusterConfigurationService.class);
     xmlEntity = mock(XmlEntity.class);
     cache = mock(InternalCache.class);
     doReturn(cache).when(command).getCache();
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java
index 63f8c37..68b717a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java
@@ -25,7 +25,7 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.jndi.JNDIInvoker;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.utils.XmlUtils;
@@ -72,7 +72,7 @@ public class DestroyJndiBindingCommandDUnitTest {
 
     // verify cluster config is updated
     locator.invoke(() -> {
-      ClusterConfigurationService ccService =
+      InternalClusterConfigurationService ccService =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration configuration = ccService.getConfiguration("cluster");
       Document document = XmlUtils.createDocumentFromXml(configuration.getCacheXmlContent());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
index f95df2e..8d517f7 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
@@ -38,7 +38,7 @@ import org.mockito.ArgumentCaptor;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.configuration.CacheConfig;
 import org.apache.geode.internal.cache.configuration.JndiBindingsType;
@@ -58,7 +58,7 @@ public class DestroyJndiBindingCommandTest {
   private DestroyJndiBindingCommand command;
   private InternalCache cache;
   private CacheConfig cacheConfig;
-  private ClusterConfigurationService ccService;
+  private InternalClusterConfigurationService ccService;
 
   private static String COMMAND = "destroy jndi-binding ";
 
@@ -68,7 +68,7 @@ public class DestroyJndiBindingCommandTest {
     command = spy(DestroyJndiBindingCommand.class);
     doReturn(cache).when(command).getCache();
     cacheConfig = mock(CacheConfig.class);
-    ccService = mock(ClusterConfigurationService.class);
+    ccService = mock(InternalClusterConfigurationService.class);
     doReturn(ccService).when(command).getSharedConfiguration();
     when(ccService.getCacheConfig(any())).thenReturn(cacheConfig);
     doCallRealMethod().when(ccService).updateCacheConfig(any(), any());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
index 99dcefb..d8fba17 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandDUnitTest.java
@@ -28,7 +28,7 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
@@ -112,7 +112,7 @@ public class DestroyRegionCommandDUnitTest {
     locator.waitTillRegionsAreReadyOnServers("/region1", 3);
 
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration group1Config = service.getConfiguration("group1");
       assertThat(group1Config.getCacheXmlContent()).contains("<region name=\"region1\">\n"
@@ -128,7 +128,7 @@ public class DestroyRegionCommandDUnitTest {
 
     // verify that all cc entries are deleted, no matter what the scope is
     locator.invoke(() -> {
-      ClusterConfigurationService service =
+      InternalClusterConfigurationService service =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Configuration group1Config = service.getConfiguration("group1");
       assertThat(group1Config.getCacheXmlContent()).doesNotContain("region1");
@@ -150,7 +150,7 @@ public class DestroyRegionCommandDUnitTest {
 
     // Make sure the region exists in the cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       assertThat(sharedConfig.getConfiguration("cluster").getCacheXmlContent())
           .contains("Customer");
@@ -162,7 +162,7 @@ public class DestroyRegionCommandDUnitTest {
 
     // make sure the region was removed from cluster config
     locator.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       assertThat(sharedConfig.getConfiguration("cluster").getCacheXmlContent())
           .doesNotContain("Customer");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandTest.java
index fa6b682..2c7a774 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyRegionCommandTest.java
@@ -35,7 +35,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
@@ -53,14 +53,14 @@ public class DestroyRegionCommandTest {
   private DestroyRegionCommand command;
   private CommandResult result;
   private CliFunctionResult result1, result2;
-  private ClusterConfigurationService ccService;
+  private InternalClusterConfigurationService ccService;
   XmlEntity xmlEntity;
 
   @Before
   public void before() throws Exception {
     xmlEntity = mock(XmlEntity.class);
     command = spy(DestroyRegionCommand.class);
-    ccService = mock(ClusterConfigurationService.class);
+    ccService = mock(InternalClusterConfigurationService.class);
     doReturn(ccService).when(command).getSharedConfiguration();
     doReturn(mock(InternalCache.class)).when(command).getCache();
 
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
index 4d786d2..e451134 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -33,7 +33,7 @@ import org.junit.rules.TemporaryFolder;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.SnapshotTestUtil;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
@@ -193,7 +193,7 @@ public class DiskStoreCommandsDUnitTest {
 
   private boolean diskStoreExistsInClusterConfig(MemberVM jmxManager) {
     boolean result = jmxManager.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       String xmlFromConfig;
       xmlFromConfig = sharedConfig.getConfiguration(GROUP).getCacheXmlContent();
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
index a2ee67e..c281752 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
@@ -27,7 +27,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
@@ -39,13 +39,13 @@ public class GfshCommandJUnitTest {
 
   private GfshCommand command;
   private Gfsh gfsh;
-  private ClusterConfigurationService clusterConfigurationService;
+  private InternalClusterConfigurationService clusterConfigurationService;
 
   @Before
   public void before() throws Exception {
     command = spy(GfshCommand.class);
     gfsh = mock(Gfsh.class);
-    clusterConfigurationService = mock(ClusterConfigurationService.class);
+    clusterConfigurationService = mock(InternalClusterConfigurationService.class);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/IndexCommandsShareConfigurationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/IndexCommandsShareConfigurationDUnitTest.java
index c52c60f..fdb93c8 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/IndexCommandsShareConfigurationDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/IndexCommandsShareConfigurationDUnitTest.java
@@ -41,7 +41,7 @@ import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.query.Index;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.cache.InternalCache;
@@ -114,7 +114,7 @@ public class IndexCommandsShareConfigurationDUnitTest {
     assertTrue(indexIsListed());
 
     locator.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       String xmlFromConfig;
       try {
@@ -132,7 +132,7 @@ public class IndexCommandsShareConfigurationDUnitTest {
     gfsh.executeAndAssertThat(createStringBuilder.toString()).statusIsSuccess();
 
     locator.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ((InternalLocator) Locator.getLocator()).getSharedConfiguration();
       String xmlFromConfig;
       try {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
index b62231d..54503ec 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfig.java
@@ -39,7 +39,7 @@ import java.util.stream.Stream;
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.geode.cache.Cache;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.DeployedJar;
@@ -94,7 +94,7 @@ public class ClusterConfig implements Serializable {
     // verify info exists in memory
     locatorVM.invoke(() -> {
       InternalLocator internalLocator = ClusterStartupRule.getLocator();
-      ClusterConfigurationService sc = internalLocator.getSharedConfiguration();
+      InternalClusterConfigurationService sc = internalLocator.getSharedConfiguration();
 
       // verify no extra configs exist in memory
       Set<String> actualGroupConfigs = sc.getConfigurationRegion().keySet();
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
index f4705db..42777ee 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
@@ -30,7 +30,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
@@ -78,7 +78,7 @@ public class ClusterConfigWithSecurityDUnitTest {
     // the second locator should inherit the first locator's security props
     locator1.invoke(() -> {
       InternalLocator locator = ClusterStartupRule.getLocator();
-      ClusterConfigurationService sc = locator.getSharedConfiguration();
+      InternalClusterConfigurationService sc = locator.getSharedConfiguration();
       Properties clusterConfigProps = sc.getConfiguration("cluster").getGemfireProperties();
       assertThat(clusterConfigProps.getProperty(SECURITY_MANAGER))
           .isEqualTo(SimpleTestSecurityManager.class.getName());
@@ -98,7 +98,7 @@ public class ClusterConfigWithSecurityDUnitTest {
 
     locator0.invoke(() -> {
       InternalLocator locator = ClusterStartupRule.getLocator();
-      ClusterConfigurationService sc = locator.getSharedConfiguration();
+      InternalClusterConfigurationService sc = locator.getSharedConfiguration();
       Properties properties = sc.getConfiguration("cluster").getGemfireProperties();
       assertThat(properties.getProperty(MCAST_PORT)).isEqualTo("0");
       assertThat(properties.getProperty(LOG_FILE_SIZE_LIMIT)).isEqualTo("8000");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigurationServiceUsingDirDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigurationServiceUsingDirDUnitTest.java
index 748d91b..1993930 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigurationServiceUsingDirDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigurationServiceUsingDirDUnitTest.java
@@ -44,7 +44,7 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
@@ -65,7 +65,7 @@ public class ClusterConfigurationServiceUsingDirDUnitTest extends JUnit4CacheTes
           return;
         }
 
-        ClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
+        InternalClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
         if (sharedConfig != null) {
           sharedConfig.destroySharedConfiguration();
         }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/SharedConfigurationTestUtils.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/SharedConfigurationTestUtils.java
index cd4f40a..9d570ec 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/SharedConfigurationTestUtils.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/SharedConfigurationTestUtils.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.management.internal.configuration;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
@@ -26,7 +26,7 @@ public class SharedConfigurationTestUtils {
     public void run() {
       InternalLocator locator = InternalLocator.getLocator();
       if (locator != null) {
-        ClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
+        InternalClusterConfigurationService sharedConfig = locator.getSharedConfiguration();
         if (sharedConfig != null) {
           sharedConfig.destroySharedConfiguration();
         }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.java
index 79cf8c8..558759f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.java
@@ -38,7 +38,7 @@ import org.xml.sax.SAXException;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.extension.Extension;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.management.internal.configuration.domain.XmlEntity;
@@ -48,7 +48,8 @@ import org.apache.geode.test.junit.categories.IntegrationTest;
 /**
  * Unit tests for {@link XmlUtils#addNewNode(Document, XmlEntity)} and
  * {@link XmlUtils#deleteNode(Document, XmlEntity)}. Simulates the
- * {@link ClusterConfigurationService} method of extracting {@link XmlEntity} from the new config
+ * {@link InternalClusterConfigurationService} method of extracting {@link XmlEntity} from the new
+ * config
  * and applying it to the current shared config.
  *
  *
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
index b44ca8d..d4e8051 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
@@ -18,7 +18,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_ARCHIVE_FILE;
-import static org.apache.geode.distributed.internal.ClusterConfigurationService.CLUSTER_CONFIG_DISK_DIR_PREFIX;
+import static org.apache.geode.distributed.internal.InternalClusterConfigurationService.CLUSTER_CONFIG_DISK_DIR_PREFIX;
 import static org.apache.geode.test.dunit.DistributedTestUtils.getAllDistributedSystemProperties;
 import static org.apache.geode.test.dunit.DistributedTestUtils.unregisterInstantiatorsInThisVM;
 import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
@@ -27,7 +27,6 @@ import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
 import static org.junit.Assert.assertNotNull;
 
 import java.io.File;
-import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.LinkedHashSet;
@@ -42,7 +41,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 
-import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.query.QueryTestUtils;
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index a34c2f3..cea463d 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -40,7 +40,7 @@ import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
 import org.apache.geode.cache.lucene.internal.repository.serializer.PrimitiveSerializer;
 import org.apache.geode.cache.lucene.internal.xml.LuceneXmlConstants;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
@@ -262,8 +262,9 @@ public class LuceneClusterConfigurationDUnitTest {
   SerializableRunnableIF verifyClusterConfiguration(boolean verifyIndexesExist) {
     return () -> {
       InternalLocator internalLocator = ClusterStartupRule.getLocator();
-      ClusterConfigurationService sc = internalLocator.getSharedConfiguration();
-      Configuration config = sc.getConfiguration(ClusterConfigurationService.CLUSTER_CONFIG);
+      InternalClusterConfigurationService sc = internalLocator.getSharedConfiguration();
+      Configuration config =
+          sc.getConfiguration(InternalClusterConfigurationService.CLUSTER_CONFIG);
       String xmlContent = config.getCacheXmlContent();
       String luceneIndex0Config = "<" + LuceneXmlConstants.PREFIX + ":" + LuceneXmlConstants.INDEX
           + " xmlns:lucene=\"" + LuceneXmlConstants.NAMESPACE + "\" " + LuceneXmlConstants.NAME
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDUnitTest.java
index 7058b03..2fe6450 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDUnitTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDUnitTest.java
@@ -29,8 +29,8 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalClusterConfigurationService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogWriterImpl;
 import org.apache.geode.management.cli.Result;
@@ -1194,7 +1194,7 @@ public class AlterRuntimeCommandDUnitTest {
     gfsh.executeAndAssertThat(command).statusIsSuccess();
 
     locator.invoke(() -> {
-      ClusterConfigurationService sharedConfig =
+      InternalClusterConfigurationService sharedConfig =
           ClusterStartupRule.getLocator().getSharedConfiguration();
       Properties properties = sharedConfig.getConfiguration("Group1").getGemfireProperties();
       assertThat(properties.get(LOG_LEVEL)).isEqualTo("fine");

-- 
To stop receiving notification emails like this one, please contact
jinmeiliao@apache.org.