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/03/05 01:58:02 UTC

incubator-geode git commit: GEODE-17: Add generic MBean creation rule and associated config annotation

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 660d3bb2b -> 3e2ae3330


GEODE-17: Add generic MBean creation rule and associated config annotation


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 3e2ae3330125dc7a4ce9e004e5dbf2a32f1e076e
Parents: 660d3bb
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Mar 4 16:57:57 2016 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Mar 4 16:57:57 2016 -0800

----------------------------------------------------------------------
 .../internal/security/ResourceConstants.java    |   2 +-
 .../CacheServerMBeanSecurityJUnitTest.java      |  64 +++-------
 .../security/JMXConnectionConfiguration.java    |  33 +++++
 .../internal/security/JSONAuthorization.java    |   2 +-
 .../internal/security/MXBeanCreationRule.java   | 126 +++++++++++++++++++
 .../internal/security/cacheServer.json          |   3 +-
 6 files changed, 181 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceConstants.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceConstants.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceConstants.java
index 398a9b3..64279f2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceConstants.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceConstants.java
@@ -20,7 +20,7 @@ public class ResourceConstants {
 
 	public static final String DEFAULT_LABEL="";
 	
-	public static final String RESORUCE_SEC_DESCRIPTOR = "resource.secDescriptor";
+	public static final String RESOURCE_SEC_DESCRIPTOR = "resource.secDescriptor";
 	public static final String RESORUCE_AUTH_ACCESSOR = "resource-auth-accessor";
 	public static final String RESORUCE_AUTHENTICATOR = "resource-authenticator";
   public static final String RESORUCE_DEFAULT_SEC_DESCRIPTOR = "resourceSecDesc.json";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
index 184823d..e9e8942 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanSecurityJUnitTest.java
@@ -16,19 +16,12 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import static com.gemstone.gemfire.management.internal.ManagementConstants.OBJECTNAME__CLIENTSERVICE_MXBEAN;
-import static javax.management.JMX.newMBeanProxy;
-import static org.assertj.core.api.Assertions.*;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.management.CacheServerMXBean;
-import com.gemstone.gemfire.management.internal.beans.CacheServerMBean;
 import com.gemstone.gemfire.util.test.TestUtil;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -37,35 +30,28 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 
-import javax.management.JMX;
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-import javax.management.remote.JMXServiceURL;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Properties;
 
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
 public class CacheServerMBeanSecurityJUnitTest {
 
   private static Cache cache;
-  private static DistributedSystem ds;
-  private JMXConnector jmxConnector;
-  private MBeanServerConnection mbeanServer;
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
   private CacheServerMXBean cacheServerMXBean;
 
+  @Rule
+  public MXBeanCreationRule<CacheServerMXBean> mxRule = new MXBeanCreationRule(jmxManagerPort, CacheServerMXBean.class);
+
   @ClassRule
   public static RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
   @BeforeClass
   public static void beforeClassSetUp() throws Exception {
-    System.setProperty(ResourceConstants.RESORUCE_SEC_DESCRIPTOR, TestUtil.getResourcePath(CacheServerMBeanSecurityJUnitTest.class, "cacheServer.json"));
+    System.setProperty(ResourceConstants.RESOURCE_SEC_DESCRIPTOR,
+        TestUtil.getResourcePath(CacheServerMBeanSecurityJUnitTest.class, "cacheServer.json"));
 
     Properties properties = new Properties();
     properties.put(DistributionConfig.NAME_NAME, CacheServerMBeanSecurityJUnitTest.class.getSimpleName());
@@ -80,16 +66,13 @@ public class CacheServerMBeanSecurityJUnitTest {
         JSONAuthorization.class.getName() + ".create");
 
     cache = new CacheFactory(properties).create();
+    cache.addCacheServer().start();
   }
 
   @Before
   public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    this.jmxConnector.close();
-    this.jmxConnector = null;
+    assertThat(cache.getCacheServers()).hasSize(1);
+    cacheServerMXBean = mxRule.getProxyMBean();
   }
 
   @AfterClass
@@ -99,27 +82,16 @@ public class CacheServerMBeanSecurityJUnitTest {
   }
 
   @Test
-  public void sanity() throws Exception {
-    createConnection("superuser", "1234567");
-    assertThat(cache.getCacheServers()).hasSize(1);
-
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testAllAccess() throws Exception {
     cacheServerMXBean.removeIndex("foo");
     cacheServerMXBean.executeContinuousQuery("bar");
   }
 
-  private void createConnection(String username, String password) throws Exception {
-    Map<String, String[]> env = new HashMap<>();
-    env.put(JMXConnector.CREDENTIALS, new String[] {username, password});
-    JMXServiceURL url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://:" + jmxManagerPort + "/jmxrmi");
-
-    this.jmxConnector = JMXConnectorFactory.connect(url, env);
-    this.mbeanServer = this.jmxConnector.getMBeanServerConnection();
-
-    cache.addCacheServer().start();
-
-    ObjectName objectNamePattern = ObjectName.getInstance("GemFire:service=CacheServer,*");
-    ObjectInstance bean = (ObjectInstance) this.mbeanServer.queryMBeans(objectNamePattern, null).toArray()[0];
-    ObjectName oName = bean.getObjectName();
-    cacheServerMXBean = JMX.newMBeanProxy(this.mbeanServer, oName, CacheServerMXBean.class);
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testNoAccess() throws Exception {
+    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JMXConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JMXConnectionConfiguration.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JMXConnectionConfiguration.java
new file mode 100644
index 0000000..fbc9aa8
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JMXConnectionConfiguration.java
@@ -0,0 +1,33 @@
+/*
+ * 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 com.gemstone.gemfire.management.internal.security;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * This annotation is intended to be used with {@link MXBeanCreationRule} in order to configure a per-test JMX
+ * connection with a specific user and password.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface JMXConnectionConfiguration {
+  String user();
+  String password();
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
index c0088d7..c1b26a1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JSONAuthorization.java
@@ -119,7 +119,7 @@ public class JSONAuthorization implements AccessControl, Authenticator {
   }
 
   private static String readDefault() throws IOException, JSONException {
-	  String str = System.getProperty(ResourceConstants.RESORUCE_SEC_DESCRIPTOR, ResourceConstants.RESORUCE_DEFAULT_SEC_DESCRIPTOR);
+	  String str = System.getProperty(ResourceConstants.RESOURCE_SEC_DESCRIPTOR, ResourceConstants.RESORUCE_DEFAULT_SEC_DESCRIPTOR);
 		File file = new File(str);
 		FileReader reader = new FileReader(file);
 		char[] buffer = new char[(int) file.length()];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MXBeanCreationRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MXBeanCreationRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MXBeanCreationRule.java
new file mode 100644
index 0000000..4662a8d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MXBeanCreationRule.java
@@ -0,0 +1,126 @@
+/*
+ * 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 com.gemstone.gemfire.management.internal.security;
+
+import org.junit.rules.ExternalResource;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class which eases the creation of MBeans for security testing. When combined with {@link JMXConnectionConfiguration}
+ * it allows for the creation of per-test connections with different user/password combinations.
+ *
+ * @param <T> The type of MBean which will be returned.
+ */
+public class MXBeanCreationRule<T> extends ExternalResource {
+
+  private final int jmxServerPort;
+  private Class<T> proxyClass;
+  private JMXConnector jmxConnector;
+  private ObjectName beanObjectName;
+  private MBeanServerConnection mbeanServer;
+
+  private static class JMXConfigurationStatement extends Statement {
+
+    private final Description description;
+    private final Statement base;
+    private final MXBeanCreationRule mxRule;
+
+    JMXConfigurationStatement(Statement base, Description description, MXBeanCreationRule rule) {
+      this.description = description;
+      this.base = base;
+      this.mxRule = rule;
+    }
+
+    @Override
+    public void evaluate() throws Throwable {
+      mxRule.createConnection(getUser(), getPassword());
+      try {
+        base.evaluate();
+      } finally {
+        mxRule.close();
+      }
+    }
+
+    private String getUser() {
+      return description.getAnnotation(JMXConnectionConfiguration.class).user();
+    }
+
+    private String getPassword() {
+      return description.getAnnotation(JMXConnectionConfiguration.class).password();
+    }
+  }
+
+  /**
+   * Rule constructor
+   * @param port The JMX server port to connect to
+   * @param proxyClass The class for which a proxy MBean will be created
+   */
+  public MXBeanCreationRule(int port, Class<T> proxyClass) {
+    this.jmxServerPort = port;
+    this.proxyClass = proxyClass;
+  }
+
+  /**
+   * Retrieve a new proxy MBean
+   * @return A new proxy MBean of the same type with which the class was constructed
+   */
+  public T getProxyMBean() {
+    return JMX.newMBeanProxy(mbeanServer, beanObjectName, proxyClass);
+  }
+
+  private void close() throws Exception{
+    jmxConnector.close();
+  }
+
+  private void createConnection(String user, String password) throws Exception {
+    Map<String, String[]> env = new HashMap<>();
+    env.put(JMXConnector.CREDENTIALS, new String[] {user, password});
+    JMXServiceURL url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://:" + jmxServerPort + "/jmxrmi");
+
+    jmxConnector = JMXConnectorFactory.connect(url, env);
+    mbeanServer = jmxConnector.getMBeanServerConnection();
+
+    ObjectName objectNamePattern = lookupObjectName();
+    ObjectInstance bean = (ObjectInstance) mbeanServer.queryMBeans(objectNamePattern, null).toArray()[0];
+    beanObjectName = bean.getObjectName();
+  }
+
+  private ObjectName lookupObjectName() throws Exception {
+    switch (proxyClass.getName()) {
+      case "com.gemstone.gemfire.management.CacheServerMXBean":
+        return(ObjectName.getInstance("GemFire:service=CacheServer,*"));
+      default:
+        throw new RuntimeException("Unknown MBean class: " + proxyClass.getName());
+    }
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    return new JMXConfigurationStatement(base, description, this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e2ae333/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
index 325ed8c..37295d5 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
@@ -4,8 +4,9 @@
       "name": "everything",
       "operationsAllowed": [
         "INDEX:DESTROY",
+        "DESTROY_INDEX",
         "LOCATE_ENTRY",
-        "QUERY:*"
+        "QUERY"
       ]
     },
     {