You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/06/14 17:40:01 UTC

geode git commit: GEODE-3062: create new SecurityService after receiving cluster config

Repository: geode
Updated Branches:
  refs/heads/develop a64332732 -> cecad6c37


GEODE-3062: create new SecurityService after receiving cluster config


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

Branch: refs/heads/develop
Commit: cecad6c37d59369a29237f7d940f297804633aa1
Parents: a643327
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Jun 9 15:23:53 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed Jun 14 10:34:28 2017 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     |  4 +++
 .../cache/ClusterConfigurationLoader.java       |  7 ++--
 .../geode/internal/cache/GemFireCacheImpl.java  | 35 +++++++++++++-------
 .../ClusterConfigWithSecurityDUnitTest.java     | 31 +++++++++++++----
 4 files changed, 55 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/cecad6c3/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 22edb6f..f406393 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -516,6 +516,10 @@ public class InternalDistributedSystem extends DistributedSystem
     return this.securityService;
   }
 
+  public void setSecurityService(SecurityService securityService) {
+    this.securityService = securityService;
+  }
+
   /**
    * Registers a listener to the system
    * 

http://git-wip-us.apache.org/repos/asf/geode/blob/cecad6c3/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 4f4881f..92cfd96 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
@@ -152,13 +152,12 @@ public class ClusterConfigurationLoader {
 
   /***
    * 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) {
+  public static void applyClusterPropertiesConfiguration(ConfigurationResponse response,
+      DistributionConfig config) {
     if (response == null || response.getRequestedConfiguration().isEmpty()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/cecad6c3/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 40df0c7..c503c40 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
@@ -77,6 +77,7 @@ import javax.transaction.TransactionManager;
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
 import org.apache.commons.lang.StringUtils;
+import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
@@ -126,7 +127,6 @@ import org.apache.geode.cache.client.PoolFactory;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.client.internal.ClientMetadataService;
 import org.apache.geode.cache.client.internal.ClientRegionFactoryImpl;
-import org.apache.geode.cache.client.internal.ConnectionImpl;
 import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.client.internal.PoolImpl;
 import org.apache.geode.cache.control.ResourceManager;
@@ -213,7 +213,6 @@ 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.security.SecurityServiceFactory;
 import org.apache.geode.internal.sequencelog.SequenceLoggerImpl;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.internal.util.concurrent.FutureResult;
@@ -325,6 +324,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   private static final Pattern DOUBLE_BACKSLASH = Pattern.compile("\\\\");
 
+  private volatile ConfigurationResponse configurationResponse;
+
   /** To test MAX_QUERY_EXECUTION_TIME option. */
   public int testMaxQueryExecutionTime = -1;
 
@@ -811,7 +812,10 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       this.system = system;
       this.dm = this.system.getDistributionManager();
 
-      this.securityService = this.system.getSecurityService();
+      this.configurationResponse = loadClusterConfig();
+
+      this.securityService = SecurityServiceFactory.create(cacheConfig, this.system.getConfig());
+      this.system.setSecurityService(this.securityService);
 
       if (!this.isClient && PoolManager.getAll().isEmpty()) {
         // We only support management on members of a distributed system
@@ -931,6 +935,18 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     } // synchronized
   }
 
+  private ConfigurationResponse loadClusterConfig() {
+    // request and check cluster configuration
+    ConfigurationResponse configurationResponse = requestSharedConfiguration();
+    deployJarsReceivedFromClusterConfiguration(configurationResponse);
+
+    // apply the cluster's properties configuration and initialize security using that configuration
+    ClusterConfigurationLoader.applyClusterPropertiesConfiguration(configurationResponse,
+        this.system.getConfig());
+
+    return configurationResponse;
+  }
+
   @Override
   public SecurityService getSecurityService() {
     return this.securityService;
@@ -1154,13 +1170,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
     ClassPathLoader.setLatestToDefault(this.system.getConfig().getDeployWorkingDir());
 
-    // request and check cluster configuration
-    ConfigurationResponse configurationResponse = requestSharedConfiguration();
-    deployJarsReceivedFromClusterConfiguration(configurationResponse);
-
-    // apply the cluster's properties configuration and initialize security using that configuration
-    ClusterConfigurationLoader.applyClusterPropertiesConfiguration(this, configurationResponse,
-        this.system.getConfig());
+    // Cluster Config request moved from here
 
     SystemMemberCacheEventProcessor.send(this, Operation.CACHE_CREATE);
     this.resourceAdvisor.initializationGate();
@@ -1184,11 +1194,11 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
     boolean completedCacheXml = false;
     try {
-      if (configurationResponse == null) {
+      if (this.configurationResponse == null) {
         // Deploy all the jars from the deploy working dir.
         ClassPathLoader.getLatest().getJarDeployer().loadPreviouslyDeployedJarsFromDisk();
       }
-      ClusterConfigurationLoader.applyClusterXmlConfiguration(this, configurationResponse,
+      ClusterConfigurationLoader.applyClusterXmlConfiguration(this, this.configurationResponse,
           this.system.getConfig());
       initializeDeclarativeCache();
       completedCacheXml = true;
@@ -1201,6 +1211,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
           // I don't want init to throw an exception that came from the close.
           // I want it to throw the original exception that came from initializeDeclarativeCache.
         }
+        this.configurationResponse = null;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/cecad6c3/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
----------------------------------------------------------------------
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 c551ca9..41e9525 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
@@ -24,6 +24,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import org.apache.commons.io.FileUtils;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.security.SimpleTestSecurityManager;
@@ -43,7 +44,10 @@ import java.util.Properties;
 
 @Category({DistributedTest.class, SecurityTest.class})
 public class ClusterConfigWithSecurityDUnitTest {
-  public String clusterConfigZipPath;
+
+  private String clusterConfigZipPath;
+  private MemberVM locator0;
+  private Properties locatorProps;
 
   @Rule
   public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
@@ -51,9 +55,6 @@ public class ClusterConfigWithSecurityDUnitTest {
   @Rule
   public GfshShellConnectionRule connector = new GfshShellConnectionRule();
 
-  MemberVM locator0;
-  Properties locatorProps;
-
   @Before
   public void before() throws Exception {
     clusterConfigZipPath = buildSecureClusterConfigZip();
@@ -64,8 +65,8 @@ public class ClusterConfigWithSecurityDUnitTest {
   }
 
   @Test
-  @Ignore("GEODE-2315")
-  public void testSecurityPropsInheritance() throws Exception {
+  @Ignore("Fails until GEODE-2315 is implemented")
+  public void testSecurityPropsInheritanceOnLocator() throws Exception {
     locatorProps.clear();
     locatorProps.setProperty(LOCATORS, "localhost[" + locator0.getPort() + "]");
     locatorProps.setProperty("security-username", "cluster");
@@ -104,6 +105,24 @@ public class ClusterConfigWithSecurityDUnitTest {
     });
   }
 
+  @Test // fails due to GEODE-3062
+  public void testSecurityPropsInheritanceOnServer() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(LOCATORS, "localhost[" + locator0.getPort() + "]");
+    serverProps.setProperty("security-username", "cluster");
+    serverProps.setProperty("security-password", "cluster");
+    MemberVM server = lsRule.startServerVM(1, serverProps);
+
+    // cluster config specifies a security-manager so integrated security should be enabled
+    server.invoke(() -> {
+      InternalCache cache = LocatorServerStartupRule.serverStarter.getCache();
+      Properties properties = cache.getDistributedSystem().getSecurityProperties();
+      assertThat(properties.getProperty(SECURITY_MANAGER))
+          .isEqualTo(SimpleTestSecurityManager.class.getName());
+      assertThat(cache.getSecurityService().isIntegratedSecurity()).isTrue();
+    });
+  }
+
   private String buildSecureClusterConfigZip() throws Exception {
     File clusterDir = lsRule.getTempFolder().newFolder("cluster");
     File clusterSubDir = new File(clusterDir, "cluster");