You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2016/10/04 17:03:26 UTC

[20/38] incubator-geode git commit: GEODE-1659: put security properties in the cluster config and applied to all the members in the cluster.

GEODE-1659: put security properties in the cluster config and applied to all the members in the cluster.

 * break down request and apply cluster configuration into individual steps so that we can apply properties before starting services, and apply cache.xml after services are started
 * added more dunit tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/e055fad0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/e055fad0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/e055fad0

Branch: refs/heads/feature/e2e-testing
Commit: e055fad08fc0eacf829d9770956958a261fb678a
Parents: 0ad1848
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Sep 23 10:40:05 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Sep 28 07:50:03 2016 -0700

----------------------------------------------------------------------
 .../internal/SharedConfiguration.java           |  65 ++++--
 .../cache/ClusterConfigurationLoader.java       | 171 +++++++++------
 .../geode/internal/cache/GemFireCacheImpl.java  | 161 +++++++++-----
 .../geode/internal/i18n/LocalizedStrings.java   |   4 +
 .../internal/SystemManagementService.java       |   8 -
 .../internal/cache/GemFireCacheImplTest.java    |  57 ++++-
 .../PDXGfshPostProcessorOnRemoteServerTest.java |  10 +-
 .../SecurityClusterConfigDUnitTest.java         | 211 +++++++++++++++++++
 .../SecurityWithoutClusterConfigDUnitTest.java  | 137 ++++++++++++
 .../security/StartServerAuthorizationTest.java  |  18 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   3 -
 11 files changed, 667 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/main/java/org/apache/geode/distributed/internal/SharedConfiguration.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/SharedConfiguration.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/SharedConfiguration.java
index 2e6677f..c5bd72b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/SharedConfiguration.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/SharedConfiguration.java
@@ -16,6 +16,8 @@
  */
 package org.apache.geode.distributed.internal;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileInputStream;
@@ -83,13 +85,11 @@ import org.apache.geode.management.internal.configuration.messages.SharedConfigu
 import org.apache.geode.management.internal.configuration.utils.XmlUtils;
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-
 @SuppressWarnings({ "deprecation", "unchecked" })
 public class SharedConfiguration {
 
   private static final Logger logger = LogService.getLogger();
-  
+
   /**
    * Name of the directory where the shared configuration artifacts are stored
    */
@@ -127,7 +127,7 @@ public class SharedConfiguration {
   private final DistributedLockService sharedConfigLockingService;
 
   /**
-   * Gets or creates (if not created) shared configuration lock service 
+   * Gets or creates (if not created) shared configuration lock service
    */
   public static DistributedLockService getSharedConfigLockService(DistributedSystem ds) {
     DistributedLockService sharedConfigDls = DLockService.getServiceNamed(SHARED_CONFIG_LOCK_SERVICE_NAME);
@@ -202,7 +202,7 @@ public class SharedConfiguration {
       Configuration configuration = (Configuration) configRegion.get(group);
       if (configuration == null) {
         configuration = new Configuration(group);
-      } 
+      }
       String xmlContent = configuration.getCacheXmlContent();
       if (xmlContent == null || xmlContent.isEmpty()) {
         StringWriter sw = new StringWriter();
@@ -271,8 +271,10 @@ public class SharedConfiguration {
     } else {
       //Write out the existing configuration into the 'shared_config' directory
       //And get deployed jars from other locators.
-      lockSharedConfiguration(); 
-      try {
+      lockSharedConfiguration();
+      putSecurityPropsIntoClusterConfig(configRegion);
+
+       try {
         Set<Entry<String, Configuration>> configEntries = configRegion.entrySet();
 
         for (Entry<String, Configuration> configEntry : configEntries) {
@@ -293,6 +295,23 @@ public class SharedConfiguration {
     status.set(SharedConfigurationStatus.RUNNING);
   }
 
+  private void putSecurityPropsIntoClusterConfig(final Region<String, Configuration> configRegion) {
+    Properties securityProps =  cache.getDistributedSystem().getSecurityProperties();
+    Configuration clusterPropertiesConfig = configRegion.get(SharedConfiguration.CLUSTER_CONFIG);
+    if(clusterPropertiesConfig == null){
+      clusterPropertiesConfig = new Configuration(SharedConfiguration.CLUSTER_CONFIG);
+      configRegion.put(SharedConfiguration.CLUSTER_CONFIG, clusterPropertiesConfig);
+    }
+    // put security-manager and security-post-processor in the cluster config
+    Properties clusterProperties = clusterPropertiesConfig.getGemfireProperties();
+    if (securityProps.containsKey(SECURITY_MANAGER)) {
+      clusterProperties.setProperty(SECURITY_MANAGER, securityProps.getProperty(SECURITY_MANAGER));
+    }
+    if (securityProps.containsKey(SECURITY_POST_PROCESSOR)) {
+      clusterProperties.setProperty(SECURITY_POST_PROCESSOR, securityProps.getProperty(SECURITY_POST_PROCESSOR));
+    }
+  }
+
   /**
    * Creates a ConfigurationResponse based on the configRequest, configuration response contains the requested shared configuration
    * This method locks the SharedConfiguration
@@ -328,7 +347,7 @@ public class SharedConfiguration {
       }
 
     }
-    configResponse.setFailedToGetSharedConfig(true); 
+    configResponse.setFailedToGetSharedConfig(true);
 
     return configResponse;
   }
@@ -366,7 +385,7 @@ public class SharedConfiguration {
           configRegion.put(group, configuration);
           writeConfig(configuration);
         }
-      } 
+      }
     }
   }
 
@@ -382,7 +401,7 @@ public class SharedConfiguration {
 
       if (configuration == null) {
         configuration = new Configuration(group);
-      } 
+      }
       String xmlContent = configuration.getCacheXmlContent();
       if (xmlContent == null || xmlContent.isEmpty()) {
         StringWriter sw = new StringWriter();
@@ -638,7 +657,7 @@ public class SharedConfiguration {
           jarNames = (String[])jars[0];
           jarBytes = (byte[][]) jars[1];
           break;
-        } 
+        }
       }
     }
 
@@ -700,7 +719,7 @@ public class SharedConfiguration {
           }
         }
 
-        File [] diskDirs = {diskDir}; 
+        File [] diskDirs = {diskDir};
         cache.createDiskStoreFactory()
         .setDiskDirs(diskDirs)
         .setAutoCompact(true)
@@ -740,10 +759,10 @@ public class SharedConfiguration {
    * @param configName
    * @param configDirectory
    * @return {@link Configuration}
-   * @throws TransformerException 
-   * @throws TransformerFactoryConfigurationError 
-   * @throws ParserConfigurationException 
-   * @throws SAXException 
+   * @throws TransformerException
+   * @throws TransformerFactoryConfigurationError
+   * @throws ParserConfigurationException
+   * @throws SAXException
    */
   private Configuration readConfiguration(final String configName, final String configDirectory) throws SAXException, ParserConfigurationException, TransformerFactoryConfigurationError, TransformerException {
     Configuration configuration = new Configuration(configName);
@@ -791,11 +810,11 @@ public class SharedConfiguration {
 
   /**
    * Reads the "shared_config" directory and loads all the cache.xml, gemfire.properties and deployed jars information
-   * @return {@link Map} 
-   * @throws TransformerException 
-   * @throws TransformerFactoryConfigurationError 
-   * @throws ParserConfigurationException 
-   * @throws SAXException 
+   * @return {@link Map}
+   * @throws TransformerException
+   * @throws TransformerFactoryConfigurationError
+   * @throws ParserConfigurationException
+   * @throws SAXException
    */
   private Map<String, Configuration> readSharedConfigurationFromDisk() throws SAXException, ParserConfigurationException, TransformerFactoryConfigurationError, TransformerException {
     String[] subdirectoryNames = getSubdirectories(configDirPath);
@@ -812,7 +831,7 @@ public class SharedConfiguration {
   }
 
   /**
-   * Removes the jar files from the given directory     
+   * Removes the jar files from the given directory
    * @param dirPath Path of the configuration directory
    * @param jarNames Names of the jar files
    * @throws IOException
@@ -848,7 +867,7 @@ public class SharedConfiguration {
   }
 
   /**
-   * Writes the 
+   * Writes the
    * @param dirPath target directory , where the jar files are to be written
    * @param jarNames Array containing the name of the jar files.
    * @param jarBytes Array of byte arrays for the jar files.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
----------------------------------------------------------------------
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 5281e11..cfc85b4 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
@@ -16,6 +16,21 @@
  */
 package org.apache.geode.internal.cache;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.UnmodifiableException;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.internal.DistributionConfig;
@@ -28,19 +43,10 @@ import org.apache.geode.internal.admin.remote.DistributionLocatorId;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.net.*;
 import org.apache.geode.internal.process.ClusterConfigurationNotAvailableException;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.messages.ConfigurationRequest;
 import org.apache.geode.management.internal.configuration.messages.ConfigurationResponse;
-import org.apache.logging.log4j.Logger;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.*;
 
 public class ClusterConfigurationLoader {
   
@@ -54,6 +60,9 @@ public class ClusterConfigurationLoader {
    * @throws ClassNotFoundException 
    */
   public static void deployJarsReceivedFromClusterConfiguration(Cache cache , ConfigurationResponse response) throws IOException, ClassNotFoundException {
+    if(response == null)
+      return;
+
     String []jarFileNames = response.getJarNames();
     byte [][]jarBytes = response.getJars();
     
@@ -74,86 +83,109 @@ public class ClusterConfigurationLoader {
   }
 
   /***
-   * Apply the cache-xml based configuration on this member
+   * Apply the cache-xml cluster configuration on this member
    * @param cache Cache created for this member
    * @param response {@link ConfigurationResponse} containing the requested {@link Configuration}
-   * @param groups List of groups this member belongs to.
+   * @param config this member's config.
    */
-  public static void applyClusterConfiguration(Cache cache , ConfigurationResponse response, List<String> groups) {
+  public static void applyClusterXmlConfiguration(Cache cache , ConfigurationResponse response, DistributionConfig config) {
+    if(response == null || response.getRequestedConfiguration().isEmpty())
+      return;
+
+    List<String> groups = getGroups(config);
     Map<String, Configuration> requestedConfiguration = response.getRequestedConfiguration();
 
-    final Properties runtimeProps = new Properties();
     List<String> cacheXmlContentList = new LinkedList<String>();
 
-    if (!requestedConfiguration.isEmpty()) {
-
-      //Need to apply the properties before doing a loadCacheXml
+    // apply the cluster config first
+    Configuration clusterConfiguration = requestedConfiguration.get(SharedConfiguration.CLUSTER_CONFIG);
+    if (clusterConfiguration != null) {
+      String cacheXmlContent = clusterConfiguration.getCacheXmlContent();
+      if (!StringUtils.isBlank(cacheXmlContent)) {
+        cacheXmlContentList.add(cacheXmlContent);
+      }
+    }
 
-      Configuration clusterConfiguration = requestedConfiguration.get(SharedConfiguration.CLUSTER_CONFIG);
-      if (clusterConfiguration != null) {
-        String cacheXmlContent = clusterConfiguration.getCacheXmlContent();
+    // then apply the groups config
+    for (String group : groups) {
+      Configuration groupConfiguration = requestedConfiguration.get(group);
+      if (groupConfiguration != null) {
+        String cacheXmlContent = groupConfiguration.getCacheXmlContent();
         if (!StringUtils.isBlank(cacheXmlContent)) {
           cacheXmlContentList.add(cacheXmlContent);
         }
-        runtimeProps.putAll(clusterConfiguration.getGemfireProperties());
       }
-      
-      requestedConfiguration.remove(SharedConfiguration.CLUSTER_CONFIG);
-      for (String group : groups) {
-        Configuration groupConfiguration = requestedConfiguration.get(group);
-        if (groupConfiguration != null) {
-          String cacheXmlContent = groupConfiguration.getCacheXmlContent();
-          if (!StringUtils.isBlank(cacheXmlContent)) {
-            cacheXmlContentList.add(cacheXmlContent);
-          }
-          runtimeProps.putAll(groupConfiguration.getGemfireProperties());
+    }
+
+    // apply the requested cache xml
+    for (String cacheXmlContent : cacheXmlContentList) {
+      InputStream is = new ByteArrayInputStream(cacheXmlContent.getBytes());
+      try {
+        cache.loadCacheXml(is);
+      } finally {
+        try {
+          is.close();
+        } catch (IOException e) {
         }
       }
-      
-      DistributionConfig config = ((GemFireCacheImpl)cache).getSystem().getConfig();
-
-      Set<Object> attNames = runtimeProps.keySet();
-
-      if (!attNames.isEmpty()) {
-        for (Object attNameObj : attNames) {
-          String attName = (String) attNameObj;
-          String attValue = runtimeProps.getProperty(attName) ;
-          try {
-            config.setAttribute(attName, attValue, ConfigSource.runtime());
-          } catch (IllegalArgumentException e) {
-            logger.info(e.getMessage());
-          } catch (UnmodifiableException e) {
-            logger.info(e.getMessage());
-          }
-        }
+    }
+  }
+
+  /***
+   * Apply the gemfire properties cluster configuration on this member
+   * @param cache Cache created for this member
+   * @param response {@link ConfigurationResponse} containing the requested {@link Configuration}
+   * @param config this member's config
+   */
+  public static void applyClusterPropertiesConfiguration(Cache cache , ConfigurationResponse response, DistributionConfig config) {
+    if(response == null || response.getRequestedConfiguration().isEmpty())
+      return;
+
+    List<String> groups = getGroups(config);
+    Map<String, Configuration> requestedConfiguration = response.getRequestedConfiguration();
+
+    final Properties runtimeProps = new Properties();
+
+    // apply the cluster config first
+    Configuration clusterConfiguration = requestedConfiguration.get(SharedConfiguration.CLUSTER_CONFIG);
+    if (clusterConfiguration != null) {
+      runtimeProps.putAll(clusterConfiguration.getGemfireProperties());
+    }
+
+    // then apply the group config
+    for (String group : groups) {
+      Configuration groupConfiguration = requestedConfiguration.get(group);
+      if (groupConfiguration != null) {
+        runtimeProps.putAll(groupConfiguration.getGemfireProperties());
       }
+    }
 
-      if (!cacheXmlContentList.isEmpty()) {
-        for (String cacheXmlContent : cacheXmlContentList) {
-          InputStream is = new ByteArrayInputStream(cacheXmlContent.getBytes());
-          try {
-            cache.loadCacheXml(is);
-          } finally {
-            try {
-              is.close();
-            } catch (IOException e) {
-            }
-          }
-        }
+    Set<Object> attNames = runtimeProps.keySet();
+    for (Object attNameObj : attNames) {
+      String attName = (String) attNameObj;
+      String attValue = runtimeProps.getProperty(attName) ;
+      try {
+        config.setAttribute(attName, attValue, ConfigSource.runtime());
+      } catch (IllegalArgumentException e) {
+        logger.info(e.getMessage());
+      } catch (UnmodifiableException e) {
+        logger.info(e.getMessage());
       }
     }
   }
   
   /**
    * Request the shared configuration for group(s) from locator(s) this member is bootstrapped with. 
-   * @param groups The groups this member wants to be part of.
+   * @param config this member's configuration.
    * @return {@link ConfigurationResponse}
    * @throws ClusterConfigurationNotAvailableException 
    * @throws UnknownHostException 
    */
-  public static ConfigurationResponse requestConfigurationFromLocators(List<String> groups, List<String> locatorList) throws ClusterConfigurationNotAvailableException, UnknownHostException {
+  public static ConfigurationResponse requestConfigurationFromLocators(DistributionConfig config, List<String> locatorList) throws ClusterConfigurationNotAvailableException, UnknownHostException {
+    List<String> groups = ClusterConfigurationLoader.getGroups(config);
     ConfigurationRequest request = new ConfigurationRequest();
 
+    request.addGroups(SharedConfiguration.CLUSTER_CONFIG);
     for (String group : groups) {
       request.addGroups(group);
     }
@@ -199,16 +231,15 @@ public class ClusterConfigurationLoader {
 
     return response;
   }
-  
 
- public static List<String> getGroups(String groupString) {
-   List<String> groups = new ArrayList<String>();
-   groups.add(SharedConfiguration.CLUSTER_CONFIG);
-   if (!StringUtils.isBlank(groupString)) {
-     groups.addAll((Arrays.asList(groupString.split(","))));
-   }
-   return groups;
- }
+  private static List<String> getGroups(DistributionConfig config) {
+    String groupString = config.getGroups();
+    List<String> groups = new ArrayList<String>();
+    if (!StringUtils.isBlank(groupString)) {
+      groups.addAll((Arrays.asList(groupString.split(","))));
+    }
+    return groups;
+  }
  
  /***
   * Get the host and port information of the locators 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
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 93a9c3b..d166397 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
@@ -73,13 +73,14 @@ import javax.naming.Context;
 
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
-import org.apache.geode.redis.GeodeRedisServer;
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.ForcedDisconnectException;
 import org.apache.geode.GemFireCacheException;
+import org.apache.geode.GemFireConfigException;
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.LogWriter;
 import org.apache.geode.SystemFailure;
@@ -89,7 +90,6 @@ import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.CacheExistsException;
-import org.apache.geode.cache.CacheRuntimeException;
 import org.apache.geode.cache.CacheTransactionManager;
 import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.CacheXmlException;
@@ -139,6 +139,7 @@ import org.apache.geode.cache.wan.GatewayReceiver;
 import org.apache.geode.cache.wan.GatewayReceiverFactory;
 import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.cache.wan.GatewaySenderFactory;
+import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.DistributedLockService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
@@ -159,13 +160,13 @@ import org.apache.geode.distributed.internal.ReplyProcessor21;
 import org.apache.geode.distributed.internal.ResourceEvent;
 import org.apache.geode.distributed.internal.ResourceEventsListener;
 import org.apache.geode.distributed.internal.ServerLocation;
+import org.apache.geode.distributed.internal.SharedConfiguration;
 import org.apache.geode.distributed.internal.locks.DLockService;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.JarDeployer;
-import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
@@ -208,6 +209,7 @@ import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.process.ClusterConfigurationNotAvailableException;
+import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.internal.sequencelog.SequenceLoggerImpl;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.internal.util.concurrent.FutureResult;
@@ -216,6 +218,7 @@ import org.apache.geode.management.internal.JmxManagerAdvisee;
 import org.apache.geode.management.internal.JmxManagerAdvisor;
 import org.apache.geode.management.internal.RestAgent;
 import org.apache.geode.management.internal.beans.ManagementListener;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.messages.ConfigurationResponse;
 import org.apache.geode.memcached.GemFireMemcachedServer;
 import org.apache.geode.memcached.GemFireMemcachedServer.Protocol;
@@ -227,6 +230,7 @@ import org.apache.geode.pdx.internal.AutoSerializableManager;
 import org.apache.geode.pdx.internal.PdxInstanceFactoryImpl;
 import org.apache.geode.pdx.internal.PdxInstanceImpl;
 import org.apache.geode.pdx.internal.TypeRegistry;
+import org.apache.geode.redis.GeodeRedisServer;
 
 // @todo somebody Come up with more reasonable values for {@link #DEFAULT_LOCK_TIMEOUT}, etc.
 /**
@@ -573,6 +577,8 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
 
   private final static Boolean DISABLE_AUTO_EVICTION = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "disableAutoEviction");
 
+  private static SecurityService securityService = SecurityService.getSecurityService();
+
   static {
     // this works around jdk bug 6427854, reported in ticket #44434
     String propertyName = "sun.nio.ch.bugLevel";
@@ -944,66 +950,96 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   
   /*****
    * Request the shared configuration from the locator(s) which have the Cluster config service running
-   * Applies the shared configuration to this cache, only if its a GEMFIRE && NON-LOCATOR && NON-CLIENT cache 
    */
-  public void requestAndApplySharedConfiguration() {
+  public ConfigurationResponse requestSharedConfiguration() {
     //Request the shared configuration from the locator(s)
     final DistributionConfig config = this.system.getConfig();
 
-    if (dm instanceof DistributionManager) {
-      if (((DistributionManager) dm).getDMType() != DistributionManager.LOCATOR_DM_TYPE
-          && !isClient
-          && Locator.getLocator() == null
-          ) {
-        
-        boolean useSharedConfiguration = config.getUseSharedConfiguration();
+    if (!(dm instanceof DistributionManager))
+      return null;
 
-        if (useSharedConfiguration) {
-          Map<InternalDistributedMember, Collection<String>> scl = this.getDistributionManager().getAllHostedLocatorsWithSharedConfiguration();
+    // do nothing if this vm is/has locator or this is a client
+    if( ((DistributionManager)dm).getDMType() == DistributionManager.LOCATOR_DM_TYPE
+      || isClient
+      || Locator.getLocator() !=null )
+      return null;
 
-          //If there are no locators with Shared configuration, that means the system has been started without shared configuration 
-          //then do not make requests to the locators
-          if (!scl.isEmpty()) {
-            String groupsString = config.getGroups();
-            ConfigurationResponse response = null;
-            List<String> locatorConnectionStrings = getSharedConfigLocatorConnectionStringList();
-            
-            try {
-              response = ClusterConfigurationLoader.requestConfigurationFromLocators(ClusterConfigurationLoader.getGroups(groupsString), locatorConnectionStrings);
-
-              //log the configuration received from the locator
-              logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_RECEIVED_SHARED_CONFIGURATION_FROM_LOCATORS));
-              logger.info(response.describeConfig());
-              
-              //deploy the Jars
-              ClusterConfigurationLoader.deployJarsReceivedFromClusterConfiguration(this, response);
-
-              //Apply the xml configuration 
-              ClusterConfigurationLoader.applyClusterConfiguration(this, response, ClusterConfigurationLoader.getGroups(groupsString));
-
-            } catch (ClusterConfigurationNotAvailableException e) {
-              throw new CacheRuntimeException(LocalizedStrings.GemFireCache_SHARED_CONFIGURATION_NOT_AVAILABLE.toLocalizedString(), e) {
-                private static final long serialVersionUID = 1L;
-              };
-            } catch (IOException e) {
-              throw new CacheRuntimeException(LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION.toLocalizedString(), e) {
-                private static final long serialVersionUID = 1L;
-              };      
-            } catch (ClassNotFoundException e) {
-              throw new CacheRuntimeException(LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION.toLocalizedString(), e) {
-                private static final long serialVersionUID = 1L;
-              };
-            }
-          } else {
-            logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_NO_LOCATORS_FOUND_WITH_SHARED_CONFIGURATION));
-          }
+    Map<InternalDistributedMember, Collection<String>> scl = this.getDistributionManager().getAllHostedLocatorsWithSharedConfiguration();
+
+    //If there are no locators with Shared configuration, that means the system has been started without shared configuration
+    //then do not make requests to the locators
+    if(scl.isEmpty()) {
+      logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_NO_LOCATORS_FOUND_WITH_SHARED_CONFIGURATION));
+      return null;
+    }
+
+    String groupsString = config.getGroups();
+    ConfigurationResponse response = null;
+    List<String> locatorConnectionStrings = getSharedConfigLocatorConnectionStringList();
+
+    try {
+      response = ClusterConfigurationLoader.requestConfigurationFromLocators(system.getConfig(), locatorConnectionStrings);
+
+      //log the configuration received from the locator
+      logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_RECEIVED_SHARED_CONFIGURATION_FROM_LOCATORS));
+      logger.info(response.describeConfig());
+
+      Configuration clusterConfig = response.getRequestedConfiguration().get(SharedConfiguration.CLUSTER_CONFIG);
+      Properties clusterSecProperties = (clusterConfig==null) ? new Properties():clusterConfig.getGemfireProperties();
+
+      // If not using shared configuration, return null or throw an exception is locator is secured
+      if(!config.getUseSharedConfiguration()){
+        if (clusterSecProperties.containsKey(ConfigurationProperties.SECURITY_MANAGER)) {
+          throw new GemFireConfigException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2.toLocalizedString());
         } else {
           logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_NOT_USING_SHARED_CONFIGURATION));
+          return null;
         }
       }
+
+      Properties serverSecProperties = config.getSecurityProps();
+      //check for possible mis-configuration
+      if (isMisConfigured(clusterSecProperties, serverSecProperties, ConfigurationProperties.SECURITY_MANAGER)
+       || isMisConfigured(clusterSecProperties, serverSecProperties, ConfigurationProperties.SECURITY_POST_PROCESSOR)) {
+        throw new GemFireConfigException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toLocalizedString());
+      }
+      return response;
+
+    } catch (ClusterConfigurationNotAvailableException e) {
+      throw new GemFireConfigException(LocalizedStrings.GemFireCache_SHARED_CONFIGURATION_NOT_AVAILABLE.toLocalizedString(), e);
+    } catch (UnknownHostException e) {
+      throw new GemFireConfigException(e.getLocalizedMessage(), e);
     }
   }
-  
+
+  public void deployJarsRecevedFromClusterConfiguration(ConfigurationResponse response){
+    try{
+      ClusterConfigurationLoader.deployJarsReceivedFromClusterConfiguration(this, response);
+    } catch (IOException e) {
+      throw new GemFireConfigException(LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION.toLocalizedString(), e);
+    } catch (ClassNotFoundException e) {
+      throw new GemFireConfigException(LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION.toLocalizedString(), e);
+    }
+  }
+
+
+  // When called, clusterProps and serverProps and key could not be null
+  public static boolean isMisConfigured(Properties clusterProps, Properties serverProps, String key){
+    String clusterPropValue = clusterProps.getProperty(key);
+    String serverPropValue = serverProps.getProperty(key);
+
+    // if this server prop is not specified, this is always OK.
+    if(StringUtils.isBlank(serverPropValue))
+      return false;
+
+    // server props is not blank, but cluster props is blank, NOT OK.
+    if(StringUtils.isBlank(clusterPropValue))
+      return true;
+
+    // at this point check for eqality
+    return !clusterPropValue.equals(serverPropValue);
+  }
+
   public List<String> getSharedConfigLocatorConnectionStringList() {
     List<String> locatorConnectionStringList = new ArrayList<String>();
     
@@ -1094,13 +1130,21 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     }
     
     ClassPathLoader.setLatestToDefault();
+
+    //request and check cluster configuration
+    ConfigurationResponse configurationResponse = requestSharedConfiguration();
+    deployJarsRecevedFromClusterConfiguration(configurationResponse);
+
+    // apply the cluster's properties configuration and initialize security using that configuration
+    ClusterConfigurationLoader.applyClusterPropertiesConfiguration(this, configurationResponse, system.getConfig());
+    securityService.initSecurity(system.getConfig().getSecurityProps());
        
     SystemMemberCacheEventProcessor.send(this, Operation.CACHE_CREATE);
     this.resourceAdvisor.initializationGate();
     
     //Register function that we need to execute to fetch available REST service endpoints in DS
     FunctionService.registerFunction(new FindRestEnabledServersFunction());
-    
+
     // moved this after initializeDeclarativeCache because in the future
     // distributed system creation will not happen until we have read
     // cache.xml file.
@@ -1109,18 +1153,19 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     // processing can deliver (region creation, etc.).
     // This call may need to be moved inside initializeDeclarativeCache.
     /** Entry to GemFire Management service **/
-    this.jmxAdvisor.initializationGate();  
+    this.jmxAdvisor.initializationGate();
+
+    // this starts up the ManagementService, register and federate the internal beans
     system.handleResourceEvent(ResourceEvent.CACHE_CREATE, this);
-    
-    
+
     boolean completedCacheXml = false;
-    
+
     initializeServices();
     
     try {
       //Deploy all the jars from the deploy working dir.
       new JarDeployer(this.system.getConfig().getDeployWorkingDir()).loadPreviouslyDeployedJars();
-      requestAndApplySharedConfiguration();
+      ClusterConfigurationLoader.applyClusterXmlConfiguration(this, configurationResponse, system.getConfig());
       initializeDeclarativeCache();
       completedCacheXml = true;
     } finally {
@@ -1954,6 +1999,8 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   }
 
   public void close(String reason, Throwable systemFailureCause, boolean keepalive, boolean keepDS) {
+    securityService.close();
+
     if (isClosed()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index d341f51..8bfdd68 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -3750,6 +3750,10 @@ public class LocalizedStrings {
 
   public static StringId AbstractDistributionConfig_SSL_ENABLED_COMPONENTS_INVALID_ALIAS_OPTIONS = new StringId(6643, "The alias options for the SSL options provided seem to be invalid. Please check that all required aliases are set");
 
+  public static StringId GEMFIRE_CACHE_SECURITY_MISCONFIGURATION = new StringId(6644, "A server cannot specify its own security-manager or security-post-processor when using cluster configuration");
+  public static StringId GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2 = new StringId(6645, "A server must use cluster configuration when joining a secured cluster.");
+
+
   /** Testing strings, messageId 90000-99999 **/
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
index c42d1a2..29bbb15 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
@@ -37,8 +37,6 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.security.IntegratedSecurityService;
-import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.AlreadyRunningException;
 import org.apache.geode.management.AsyncEventQueueMXBean;
 import org.apache.geode.management.CacheServerMXBean;
@@ -68,8 +66,6 @@ import org.apache.geode.management.membership.MembershipListener;
 public final class SystemManagementService extends BaseManagementService {
   private static final Logger logger = LogService.getLogger();
 
-  private SecurityService securityService = IntegratedSecurityService.getSecurityService();
-
   /**
    * The concrete implementation of DistributedSystem that provides
    * internal-only functionality.
@@ -154,7 +150,6 @@ public final class SystemManagementService extends BaseManagementService {
     this.jmxAdapter = new MBeanJMXAdapter();      
     this.repo = new ManagementResourceRepo();
 
-    this.securityService.initSecurity(system.getConfig().getSecurityProps());
 
     this.notificationHub = new NotificationHub(repo);
     if (system.getConfig().getJmxManager()) {
@@ -275,9 +270,6 @@ public final class SystemManagementService extends BaseManagementService {
         this.agent.stopAgent();
       }
 
-      // clean out Shiro's thread local content
-      this.securityService.close();
-
       getGemFireCacheImpl().getJmxManagerAdvisor().broadcastChange();
       instances.remove(cache);
       localManager  = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
index 597a977..09923f9 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
@@ -19,12 +19,12 @@ package org.apache.geode.internal.cache;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import java.util.Properties;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.junit.After;
-import org.junit.Before;
+import com.jayway.awaitility.Awaitility;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -32,7 +32,6 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.pdx.internal.TypeRegistry;
 import org.apache.geode.test.fake.Fakes;
 import org.apache.geode.test.junit.categories.UnitTest;
-import com.jayway.awaitility.Awaitility;
 
 @Category(UnitTest.class)
 public class GemFireCacheImplTest {
@@ -64,4 +63,56 @@ public class GemFireCacheImplTest {
       gfc.close();
     }
   }
+
+  @Test
+  public void testIsMisConfigured(){
+    Properties clusterProps = new Properties();
+    Properties serverProps = new Properties();
+
+    // both does not have the key
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+
+    //cluster has the key, not the server
+    clusterProps.setProperty("key", "value");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+    clusterProps.setProperty("key", "");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+
+    // server has the key, not the cluster
+    clusterProps.clear();
+    serverProps.clear();
+    serverProps.setProperty("key", "value");
+    assertTrue(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+    serverProps.setProperty("key", "");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+
+    // server has the key, not the cluster
+    clusterProps.clear();
+    serverProps.clear();
+    clusterProps.setProperty("key", "");
+    serverProps.setProperty("key", "value");
+    assertTrue(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+    serverProps.setProperty("key", "");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+
+    // server and cluster has the same value
+    clusterProps.clear();
+    serverProps.clear();
+    clusterProps.setProperty("key", "value");
+    serverProps.setProperty("key", "value");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+    clusterProps.setProperty("key", "");
+    serverProps.setProperty("key", "");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+
+    // server and cluster has the different value
+    clusterProps.clear();
+    serverProps.clear();
+    clusterProps.setProperty("key", "value1");
+    serverProps.setProperty("key", "value2");
+    assertTrue(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+    clusterProps.setProperty("key", "value1");
+    serverProps.setProperty("key", "");
+    assertFalse(GemFireCacheImpl.isMisConfigured(clusterProps, serverProps, "key"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/java/org/apache/geode/security/PDXGfshPostProcessorOnRemoteServerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PDXGfshPostProcessorOnRemoteServerTest.java b/geode-core/src/test/java/org/apache/geode/security/PDXGfshPostProcessorOnRemoteServerTest.java
index 34043e8..c981a12 100644
--- a/geode-core/src/test/java/org/apache/geode/security/PDXGfshPostProcessorOnRemoteServerTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/PDXGfshPostProcessorOnRemoteServerTest.java
@@ -25,6 +25,10 @@ import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import com.jayway.awaitility.Awaitility;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.Region;
@@ -48,9 +52,6 @@ import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 @Category({ DistributedTest.class, SecurityTest.class })
 public class PDXGfshPostProcessorOnRemoteServerTest extends JUnit4DistributedTestCase {
@@ -89,9 +90,10 @@ public class PDXGfshPostProcessorOnRemoteServerTest extends JUnit4DistributedTes
       Properties props = new Properties();
       props.setProperty(MCAST_PORT, "0");
       props.setProperty(LOCATORS, locators);
-      props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
       props.setProperty(SampleSecurityManager.SECURITY_JSON, "org/apache/geode/management/internal/security/clientServer.json");
+      props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
       props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
+      props.setProperty(USE_CLUSTER_CONFIGURATION, "true");
 
       // the following are needed for peer-to-peer authentication
       props.setProperty("security-username", "super-user");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
new file mode 100644
index 0000000..395d73e
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityClusterConfigDUnitTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Java6Assertions.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.GemFireConfigException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.security.templates.SampleSecurityManager;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+
+public class SecurityClusterConfigDUnitTest extends JUnit4DistributedTestCase {
+
+  private int locatorPort = 0;
+
+  @Rule
+  public transient TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void before() throws Exception {
+    IgnoredException.addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toString());
+    IgnoredException.addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2.toString());
+    File locatorFile = temporaryFolder.newFile("locator.log");
+    final Host host = Host.getHost(0);
+    VM locator = host.getVM(0);
+    // set up locator with security
+    this.locatorPort = locator.invoke(() -> {
+      Properties props = new Properties();
+      props.setProperty(SampleSecurityManager.SECURITY_JSON, "org/apache/geode/management/internal/security/cacheServer.json");
+      props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
+      props.setProperty(MCAST_PORT, "0");
+      props.put(JMX_MANAGER, "false");
+      props.put(JMX_MANAGER_START, "false");
+      props.put(JMX_MANAGER_PORT, 0);
+      props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
+      Locator lc = Locator.startLocatorAndDS(locatorPort, locatorFile, props);
+      return lc.getPort();
+    });
+  }
+
+  @After
+  public void after() {
+    IgnoredException.removeAllExpectedExceptions();
+  }
+
+  @Test
+  public void testStartServerWithClusterConfig() throws Exception {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty("use-cluster-configuration", "true");
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+    assertEquals(2, ds.getSecurityProperties().size());
+
+    CacheFactory.create(ds);
+
+    // after cache is created, we got the security props passed in by cluster config
+    Properties secProps = ds.getSecurityProperties();
+    assertEquals(4, secProps.size());
+    assertTrue(secProps.containsKey("security-manager"));
+    assertTrue(secProps.containsKey("security-post-processor"));
+  }
+
+  @Test
+  public void testStartServerWithSameSecurityManager() throws Exception {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty("use-cluster-configuration", "true");
+    props.setProperty(SampleSecurityManager.SECURITY_JSON, "org/apache/geode/management/internal/security/cacheServer.json");
+    props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+
+    CacheFactory.create(ds);
+
+    // after cache is created, we got the security props passed in by cluster config
+    Properties secProps = ds.getSecurityProperties();
+    assertTrue(secProps.containsKey("security-manager"));
+    assertTrue(secProps.containsKey("security-post-processor"));
+  }
+
+  @Test
+  public void serverWithDifferentSecurityManagerShouldThrowException() {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty("security-manager", "mySecurityManager");
+    props.setProperty("use-cluster-configuration", "true");
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+
+    assertThatThrownBy(() -> CacheFactory.create(ds)).isInstanceOf(GemFireConfigException.class)
+                                                     .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION
+                                                       .toLocalizedString());
+
+  }
+
+  @Test
+  public void serverWithDifferentPostProcessorShouldThrowException() {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty(SECURITY_POST_PROCESSOR, "this-is-not-ok");
+    props.setProperty("use-cluster-configuration", "true");
+
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+
+    assertThatThrownBy(() -> CacheFactory.create(ds)).isInstanceOf(GemFireConfigException.class)
+                                                     .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION
+                                                       .toLocalizedString());
+
+  }
+
+
+  @Test
+  public void serverConnectingToSecuredLocatorMustUseClusterConfig() {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty("security-manager", "mySecurityManager");
+    props.setProperty("use-cluster-configuration", "false");
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+
+    assertThatThrownBy(() -> CacheFactory.create(ds)).isInstanceOf(GemFireConfigException.class)
+                                                     .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2
+                                                       .toLocalizedString());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
new file mode 100644
index 0000000..877f057
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
@@ -0,0 +1,137 @@
+/*
+ * 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.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Java6Assertions.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.GemFireConfigException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.security.templates.SampleSecurityManager;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+
+public class SecurityWithoutClusterConfigDUnitTest extends JUnit4DistributedTestCase {
+
+  private int locatorPort = 0;
+
+  @Rule
+  public transient TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void before() throws Exception {
+    IgnoredException.addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION.toString());
+    IgnoredException.addIgnoredException(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION_2.toString());
+    File locatorFile = temporaryFolder.newFile("locator.log");
+    final Host host = Host.getHost(0);
+    VM locator = host.getVM(0);
+    // set up locator with security
+    this.locatorPort = locator.invoke(() -> {
+      Properties props = new Properties();
+      props.setProperty(MCAST_PORT, "0");
+      props.put(JMX_MANAGER, "false");
+      props.put(JMX_MANAGER_START, "false");
+      props.put(JMX_MANAGER_PORT, 0);
+      ;
+      Locator lc = Locator.startLocatorAndDS(locatorPort, locatorFile, props);
+      return lc.getPort();
+    });
+  }
+
+  @After
+  public void after() {
+    IgnoredException.removeAllExpectedExceptions();
+  }
+
+  @Test
+  public void serverShouldBeAllowedToStartWithSecurityIfNotUsingClusterConfig() throws Exception {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty(SampleSecurityManager.SECURITY_JSON, "org/apache/geode/management/internal/security/cacheServer.json");
+    props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
+    props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
+
+    props.setProperty("use-cluster-configuration", "false");
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+    assertEquals(5, ds.getSecurityProperties().size());
+
+    CacheFactory.create(ds);
+
+    // after cache is created, we got the security props passed in by cluster config
+    Properties secProps = ds.getSecurityProperties();
+    assertEquals(5, secProps.size());
+    assertEquals(SampleSecurityManager.class.getName(), secProps.getProperty("security-manager"));
+    assertEquals(PDXPostProcessor.class.getName(), secProps.getProperty("security-post-processor"));
+  }
+
+
+  @Test
+  public void serverShouldNotBeAllowedToStartWithSecurityIfUsingClusterConfig() {
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, locators);
+
+    // the following are needed for peer-to-peer authentication
+    props.setProperty("security-username", "cluster-manager");
+    props.setProperty("security-password", "1234567");
+    props.setProperty("security-manager", "mySecurityManager");
+    props.setProperty("use-cluster-configuration", "true");
+
+    // initial security properties should only contain initial set of values
+    InternalDistributedSystem ds = getSystem(props);
+
+    assertThatThrownBy(() -> CacheFactory.create(ds)).isInstanceOf(GemFireConfigException.class)
+                                                     .hasMessage(LocalizedStrings.GEMFIRE_CACHE_SECURITY_MISCONFIGURATION
+                                                       .toLocalizedString());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java b/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
index 8468664..dae062a 100644
--- a/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/StartServerAuthorizationTest.java
@@ -23,17 +23,17 @@ import static org.assertj.core.api.Assertions.*;
 import java.io.File;
 import java.util.Properties;
 
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.security.templates.SampleSecurityManager;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 @Category({ DistributedTest.class, SecurityTest.class })
 public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
@@ -44,19 +44,17 @@ public class StartServerAuthorizationTest extends JUnit4DistributedTestCase {
     final Host host = Host.getHost(0);
     VM locator = host.getVM(0);
     // set up locator with security
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-    this.locatorPort = ports[0];
-    int jmxPort = ports[1];
-    locator.invoke(()->{
+    this.locatorPort = locator.invoke(()->{
       Properties props = new Properties();
       props.setProperty(SampleSecurityManager.SECURITY_JSON, "org/apache/geode/management/internal/security/cacheServer.json");
       props.setProperty(SECURITY_MANAGER, SampleSecurityManager.class.getName());
       props.setProperty(MCAST_PORT, "0");
       props.put(JMX_MANAGER, "true");
       props.put(JMX_MANAGER_START, "true");
-      props.put(JMX_MANAGER_PORT, jmxPort+"");
+      props.put(JMX_MANAGER_PORT, 0);
       props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
-      Locator.startLocatorAndDS(locatorPort, new File("locator.log"), props);
+      Locator lc = Locator.startLocatorAndDS(locatorPort, new File("locator.log"), props);
+      return lc.getPort();
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e055fad0/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
index 8ca0b5b..fffe7d9 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
@@ -276,9 +276,6 @@ org/apache/geode/internal/cache/FilterProfile$interestType,false
 org/apache/geode/internal/cache/FilterProfile$operationType,false
 org/apache/geode/internal/cache/ForceReattemptException,true,-595988965679204903,hasHash:boolean,keyHash:int
 org/apache/geode/internal/cache/ForceableLinkedBlockingQueue,true,-6903933977591709194,capacity:int,count:java/util/concurrent/atomic/AtomicInteger,notEmpty:java/util/concurrent/locks/Condition,notFull:java/util/concurrent/locks/Condition,putLock:java/util/concurrent/locks/ReentrantLock,takeLock:java/util/concurrent/locks/ReentrantLock
-org/apache/geode/internal/cache/GemFireCacheImpl$3,true,1,this$0:org/apache/geode/internal/cache/GemFireCacheImpl
-org/apache/geode/internal/cache/GemFireCacheImpl$4,true,1,this$0:org/apache/geode/internal/cache/GemFireCacheImpl
-org/apache/geode/internal/cache/GemFireCacheImpl$5,true,1,this$0:org/apache/geode/internal/cache/GemFireCacheImpl
 org/apache/geode/internal/cache/IdentityArrayList,true,449125332499184497,size:int,wrapped:boolean
 org/apache/geode/internal/cache/IncomingGatewayStatus,true,-4579815367602658353,_memberId:java/lang/String,_socketAddress:java/net/InetAddress,_socketPort:int
 org/apache/geode/internal/cache/InitialImageOperation$GIIStatus,false