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

incubator-geode git commit: GEODE-17: add more tests

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 8ab9c82c7 -> 9801683b2


GEODE-17: add more 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/9801683b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/9801683b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/9801683b

Branch: refs/heads/feature/GEODE-17-2
Commit: 9801683b2880c2cae17fdbdbb477b20bfce80920
Parents: 8ab9c82
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Mar 9 21:50:32 2016 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Mar 9 21:50:32 2016 -0800

----------------------------------------------------------------------
 .../CacheServerMBeanSecurityJUnitTest.java      | 89 +++++++++-----------
 .../JsonAuthorizationMBeanServerStartRule.java  | 61 ++++++++++++++
 .../security/MBeanServerConnectionRule.java     | 43 ++++++++--
 .../security/MemberMBeanSecurityJUnitTest.java  | 78 +++++++++++++++++
 .../internal/security/cacheServer.json          |  8 +-
 5 files changed, 224 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9801683b/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 ec47a09..11feec5 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,71 +16,42 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.management.CacheServerMXBean;
-import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectInstance;
 import javax.management.ObjectName;
-import java.util.Properties;
+import java.io.IOException;
+import java.util.Set;
 
 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 int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
   private CacheServerMXBean cacheServerMXBean;
-
-  @Rule
-  public MBeanServerConnectionRule<CacheServerMXBean> mxRule = new MBeanServerConnectionRule(jmxManagerPort);
+  private MBeanServerConnection con;
 
   @ClassRule
-  public static RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @BeforeClass
-  public static void beforeClassSetUp() throws Exception {
-    Properties properties = new Properties();
-    properties.put(DistributionConfig.NAME_NAME, CacheServerMBeanSecurityJUnitTest.class.getSimpleName());
-    properties.put(DistributionConfig.LOCATORS_NAME, "");
-    properties.put(DistributionConfig.MCAST_PORT_NAME, "0");
-    properties.put(DistributionConfig.JMX_MANAGER_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
-    properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
-    properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
-    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
-    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
-        JSONAuthorization.class.getName() + ".create");
-    JSONAuthorization.setUpWithJsonFile("cacheServer.json");
-
-    cache = new CacheFactory(properties).create();
-    cache.addCacheServer().start();
-  }
+  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule<CacheServerMXBean> connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
 
   @Before
   public void setUp() throws Exception {
-    assertThat(cache.getCacheServers()).hasSize(1);
-    cacheServerMXBean = mxRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
-  }
-
-  @AfterClass
-  public static void afterClassTearDown() throws Exception {
-    cache.close();
-    cache = null;
+    //assertThat(cache.getCacheServers()).hasSize(1);
+    cacheServerMXBean = connectionRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
+    con = connectionRule.getMBeanServerConnection();
   }
 
-
-
   /**
    * No user can call createBean or unregisterBean
    */
@@ -88,11 +59,11 @@ public class CacheServerMBeanSecurityJUnitTest {
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testNoAccessWithWhoever() throws Exception{
     assertThatThrownBy(
-        () -> mxRule.getMBeanServerConnection().createMBean("FakeClassName", new ObjectName("GemFire", "name", "foo"))
+        () -> con.createMBean("FakeClassName", new ObjectName("GemFire", "name", "foo"))
     ).isInstanceOf(SecurityException.class);
 
     assertThatThrownBy(
-        () -> mxRule.getMBeanServerConnection().unregisterMBean(new ObjectName("GemFire", "name", "foo"))
+        () -> con.unregisterMBean(new ObjectName("GemFire", "name", "foo"))
     ).isInstanceOf(SecurityException.class);
   }
 
@@ -100,9 +71,14 @@ public class CacheServerMBeanSecurityJUnitTest {
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo");
-    cacheServerMXBean.executeContinuousQuery("bar");
-    cacheServerMXBean.fetchLoadProbe();
+    cacheServerMXBean.removeIndex("foo"); // "DESTROY_INDEX",
+    cacheServerMXBean.executeContinuousQuery("bar"); //QUERY
+    cacheServerMXBean.fetchLoadProbe(); //LIST_DS
+    cacheServerMXBean.getActiveCQCount(); //LIST_DS
+    cacheServerMXBean.stopContinuousQuery("bar"); //STOP_CONTINUOUS_QUERY
+    cacheServerMXBean.closeAllContinuousQuery("bar"); //STOP_CONTINUOUS_QUERY
+    cacheServerMXBean.isRunning(); //LIST_DS
+    cacheServerMXBean.showClientQueueDetails("foo"); //LIST_DS
   }
 
   @Test
@@ -118,5 +94,24 @@ public class CacheServerMBeanSecurityJUnitTest {
   public void testNoAccess() throws Exception {
     assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
     assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.fetchLoadProbe()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.getActiveCQCount()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.stopContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.closeAllContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.isRunning()).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> cacheServerMXBean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class);
+  }
+
+  /*
+   * looks like everyone can query for beans, but the AccessControlMXBean is filtered from the result
+   */
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testQueryBean() throws MalformedObjectNameException, IOException {
+    Set<ObjectInstance> objects = con.queryMBeans(ObjectName.getInstance(ResourceConstants.OBJECT_NAME_ACCESSCONTROL), null);
+    assertThat(objects.size()).isEqualTo(0); // no AccessControlMBean in the query result
+
+    objects = con.queryMBeans(ObjectName.getInstance("GemFire:service=CacheServer,*"), null);
+    assertThat(objects.size()).isEqualTo(1);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9801683b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
new file mode 100644
index 0000000..2ac3fa1
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/JsonAuthorizationMBeanServerStartRule.java
@@ -0,0 +1,61 @@
+/*
+ * 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 com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import org.junit.rules.ExternalResource;
+
+import java.util.Properties;
+
+public class JsonAuthorizationMBeanServerStartRule extends ExternalResource {
+  private Cache cache;
+  private int jmxManagerPort;
+  private String jsonFile;
+
+  public JsonAuthorizationMBeanServerStartRule(int jmxManagerPort, String jsonFile){
+    this.jmxManagerPort = jmxManagerPort;
+    this.jsonFile = jsonFile;
+  }
+
+  protected void before() throws Throwable {
+    Properties properties = new Properties();
+    properties.put(DistributionConfig.NAME_NAME, JsonAuthorizationMBeanServerStartRule.class.getSimpleName());
+    properties.put(DistributionConfig.LOCATORS_NAME, "");
+    properties.put(DistributionConfig.MCAST_PORT_NAME, "0");
+    properties.put(DistributionConfig.JMX_MANAGER_NAME, "true");
+    properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+    properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
+    properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
+    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
+    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
+        JSONAuthorization.class.getName() + ".create");
+    JSONAuthorization.setUpWithJsonFile(jsonFile);
+
+    cache = new CacheFactory(properties).create();
+    cache.addCacheServer().start();
+  }
+
+  /**
+   * Override to tear down your specific external resource.
+   */
+  protected void after() {
+    cache.close();
+    cache = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9801683b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
index bb5feed..ce3b63b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MBeanServerConnectionRule.java
@@ -24,12 +24,15 @@ import javax.management.MBeanServerConnection;
 import javax.management.MalformedObjectNameException;
 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.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Class which eases the creation of MBeans for security testing. When combined with {@link JMXConnectionConfiguration}
@@ -41,7 +44,7 @@ public class MBeanServerConnectionRule<T> extends DescribedExternalResource {
 
   private final int jmxServerPort;
   private JMXConnector jmxConnector;
-  private MBeanServerConnection mbeanServer;
+  private MBeanServerConnection con;
 
 
   /**
@@ -56,14 +59,40 @@ public class MBeanServerConnectionRule<T> extends DescribedExternalResource {
    * Retrieve a new proxy MBean
    * @return A new proxy MBean of the same type with which the class was constructed
    */
-  public T getProxyMBean(Class<T> proxyClass, String beanName) throws MalformedObjectNameException, IOException {
-    ObjectInstance bean = (ObjectInstance) mbeanServer.queryMBeans(ObjectName.getInstance(beanName), null).toArray()[0];
-    return JMX.newMXBeanProxy(mbeanServer, bean.getObjectName(), proxyClass);
+  public T getProxyMBean(Class<T> proxyClass, String beanQueryName) throws MalformedObjectNameException, IOException {
+    ObjectName name = null;
+    QueryExp query = null;
+
+    if(proxyClass != null){
+      query = Query.isInstanceOf(Query.value(proxyClass.getName()));
+    }
+
+    if(beanQueryName != null){
+      name = ObjectName.getInstance(beanQueryName);
+    }
+
+    Set<ObjectInstance> beans = con.queryMBeans(name, query);
+    if(beans.size() != 1){
+      throw new RuntimeException("failed to find only one instance of "+proxyClass.getName()+" with name "+beanQueryName);
+    }
+    return JMX.newMXBeanProxy(con, ((ObjectInstance)beans.toArray()[0]).getObjectName(), proxyClass);
+  }
+
+  /**
+   * Retrieve a new proxy MBean
+   * @return A new proxy MBean of the same type with which the class was constructed
+   */
+  public T getProxyMBean(Class<T> proxyClass) throws MalformedObjectNameException, IOException {
+    return getProxyMBean(proxyClass, null);
+  }
+
+  public T getProxyMBean(String beanQueryName) throws MalformedObjectNameException, IOException {
+    return getProxyMBean(null, beanQueryName);
   }
 
 
   public MBeanServerConnection getMBeanServerConnection() throws IOException {
-    return jmxConnector.getMBeanServerConnection();
+    return con;
   }
 
   protected void before(Description description) throws Throwable {
@@ -77,7 +106,7 @@ public class MBeanServerConnectionRule<T> extends DescribedExternalResource {
     JMXServiceURL url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://:" + jmxServerPort + "/jmxrmi");
 
     jmxConnector = JMXConnectorFactory.connect(url, env);
-    mbeanServer = jmxConnector.getMBeanServerConnection();
+    con = jmxConnector.getMBeanServerConnection();
   }
 
   /**
@@ -86,7 +115,7 @@ public class MBeanServerConnectionRule<T> extends DescribedExternalResource {
   protected void after(Description description) throws Throwable {
     jmxConnector.close();
     jmxConnector = null;
-    mbeanServer = null;
+    con = null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9801683b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
new file mode 100644
index 0000000..2a2101b
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.management.MemberMXBean;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+
+import javax.management.MBeanServerConnection;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class MemberMBeanSecurityJUnitTest {
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  private MemberMXBean bean;
+  private MBeanServerConnection con;
+
+  @ClassRule
+  public static JsonAuthorizationMBeanServerStartRule serverRule = new JsonAuthorizationMBeanServerStartRule(jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule<MemberMXBean> connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+  @Before
+  public void setUp() throws Exception {
+    bean = connectionRule.getProxyMBean(MemberMXBean.class);
+    con = connectionRule.getMBeanServerConnection();
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testAllAccess() throws Exception {
+    bean.shutDownMember();  //SHUTDOWN
+    bean.compactAllDiskStores(); //COMPACT_DISKSTORE
+    bean.createManager(); //CREATE_MANAGER
+    bean.fetchJvmThreads(); //LIST_DS
+    bean.getName(); //LIST_DS
+    bean.getDiskStores(); //LIST_DS
+    bean.hasGatewayReceiver(); //LIST_DS
+    bean.isCacheServer(); //LIST_DS
+    bean.isServer(); //LIST_DS
+    bean.listConnectedGatewayReceivers(); //LIST_DS
+    bean.processCommand("create region --name=Region_A"); //CREATE_REGION
+    bean.showJVMMetrics(); //LIST_DS
+    bean.status(); //LIST_DS
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "user", password = "1234567")
+  public void testSomeAccess() throws Exception {
+
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testNoAccess() throws Exception {
+    assertThatThrownBy(() -> bean.shutDownMember()).isInstanceOf(SecurityException.class);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9801683b/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 7de666c..386f785 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
@@ -6,7 +6,13 @@
         "LIST_DS",
         "DESTROY_INDEX",
         "LOCATE_ENTRY",
-        "QUERY"
+        "QUERY",
+        "STOP_CONTINUOUS_QUERY",
+        "SHUTDOWN",
+        "COMPACT_DISKSTORE",
+        "CREATE_MANAGER",
+        "CREATE_REGION",
+        "some-nonsense"
       ]
     },
     {