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/16 19:01:53 UTC

incubator-geode git commit: GEODE-17: add DiskStoreMXBean security test and some renaming.

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 3c562c8b6 -> d73606c8e


GEODE-17:  add DiskStoreMXBean security test and some renaming.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: d73606c8e7971198f72885e3e6ff9954d3fbd138
Parents: 3c562c8
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Mar 16 11:01:35 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Mar 16 11:01:35 2016 -0700

----------------------------------------------------------------------
 .../gemfire/management/DiskStoreMXBean.java     |  7 +-
 ...CacheServerMBeanAuthenticationJUnitTest.java | 26 +++---
 .../CacheServerMBeanAuthorizationJUnitTest.java | 42 +++++-----
 .../DiskStoreMXBeanSecurityJUnitTest.java       | 83 ++++++++++++++++++++
 .../LockServiceMBeanAuthorizationJUnitTest.java |  3 +-
 .../internal/security/cacheServer.json          |  3 +
 6 files changed, 124 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
index e23bc12..f2ae00b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DiskStoreMXBean.java
@@ -17,7 +17,6 @@
 package com.gemstone.gemfire.management;
 
 import com.gemstone.gemfire.cache.DiskStore;
-import com.gemstone.gemfire.cache.operations.OperationContext;
 import com.gemstone.gemfire.management.internal.security.Resource;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 
@@ -162,7 +161,7 @@ public interface DiskStoreMXBean {
    * compaction is true then the application will wait for the other op-logs to
    * be compacted and additional space is available.
    */
-  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationContext.OperationCode.ROLL)
+  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationCode.ROLL)
   public void forceRoll();
 
   /**
@@ -209,7 +208,7 @@ public interface DiskStoreMXBean {
    * 
    * @param warningPercent the warning percent
    */
-  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationCode.SET_DISK_USAGE_WARNING)
+  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationCode.ALTER)
   public void setDiskUsageWarningPercentage(float warningPercent);
   
   /**
@@ -217,6 +216,6 @@ public interface DiskStoreMXBean {
    * 
    * @param criticalPercent the critical percent
    */
-  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationCode.SET_DISK_USAGE_CRITICAL)
+  @ResourceOperation(resource = Resource.DISKSTORE, operation = OperationCode.ALTER)
   public void setDiskUsageCriticalPercentage(float criticalPercent);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
index 4cf7857..d099db1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthenticationJUnitTest.java
@@ -29,7 +29,7 @@ import org.junit.experimental.categories.Category;
 public class CacheServerMBeanAuthenticationJUnitTest {
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
-  private CacheServerMXBean cacheServerMXBean;
+  private CacheServerMXBean bean;
 
   @ClassRule
   public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
@@ -40,27 +40,27 @@ public class CacheServerMBeanAuthenticationJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    cacheServerMXBean = connectionRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
+    bean = connectionRule.getProxyMBean(CacheServerMXBean.class, "GemFire:service=CacheServer,*");
   }
 
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo"); // "INDEX:DESTROY",
-    cacheServerMXBean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
-    cacheServerMXBean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
-    cacheServerMXBean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
-    cacheServerMXBean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
-    cacheServerMXBean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
+    bean.removeIndex("foo"); // "INDEX:DESTROY",
+    bean.executeContinuousQuery("bar"); // CONTNUOUS_QUERY:EXECUTE
+    bean.fetchLoadProbe(); // DISTRIBUTED_SYSTEM:LIST_DS
+    bean.getActiveCQCount(); // DISTRIBUTED_SYSTEM:LIST_DS
+    bean.stopContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    bean.closeAllContinuousQuery("bar"); // CONTINUOUS_QUERY:STOP
+    bean.isRunning(); // DISTRIBUTED_SYSTEM:LIST_DS
+    bean.showClientQueueDetails("foo"); // DISTRIBUTED_SYSTEM:LIST_DS
   }
 
   @Test
   @JMXConnectionConfiguration(user = "user", password = "1234567")
   public void testSomeAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo");
-    cacheServerMXBean.executeContinuousQuery("bar");
-    cacheServerMXBean.fetchLoadProbe();
+    bean.removeIndex("foo");
+    bean.executeContinuousQuery("bar");
+    bean.fetchLoadProbe();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
index 7fa36a3..16cbb21 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanAuthorizationJUnitTest.java
@@ -31,7 +31,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy;
 public class CacheServerMBeanAuthorizationJUnitTest {
   private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
-  private CacheServerMXBean cacheServerMXBean;
+  private CacheServerMXBean bean;
 
   @ClassRule
   public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
@@ -42,40 +42,40 @@ public class CacheServerMBeanAuthorizationJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    cacheServerMXBean = connectionRule.getProxyMBean(CacheServerMXBean.class);
+    bean = connectionRule.getProxyMBean(CacheServerMXBean.class);
   }
 
   @Test
   @JMXConnectionConfiguration(user = "superuser", password = "1234567")
   public void testAllAccess() throws Exception {
-    cacheServerMXBean.removeIndex("foo");
-    cacheServerMXBean.executeContinuousQuery("bar");
-    cacheServerMXBean.fetchLoadProbe();
-    cacheServerMXBean.getActiveCQCount();
-    cacheServerMXBean.stopContinuousQuery("bar");
-    cacheServerMXBean.closeAllContinuousQuery("bar");
-    cacheServerMXBean.isRunning();
-    cacheServerMXBean.showClientQueueDetails("foo");
+    bean.removeIndex("foo");
+    bean.executeContinuousQuery("bar");
+    bean.fetchLoadProbe();
+    bean.getActiveCQCount();
+    bean.stopContinuousQuery("bar");
+    bean.closeAllContinuousQuery("bar");
+    bean.isRunning();
+    bean.showClientQueueDetails("foo");
   }
 
   @Test
   @JMXConnectionConfiguration(user = "user", password = "1234567")
   public void testSomeAccess() throws Exception {
-    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class);
-    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
-    cacheServerMXBean.fetchLoadProbe();
+    assertThatThrownBy(() -> bean.removeIndex("foo")).isInstanceOf(SecurityException.class);
+    assertThatThrownBy(() -> bean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class);
+    bean.fetchLoadProbe();
   }
 
   @Test
   @JMXConnectionConfiguration(user = "stranger", password = "1234567")
   public void testNoAccess() throws Exception {
-    assertThatThrownBy(() -> cacheServerMXBean.removeIndex("foo")).isInstanceOf(SecurityException.class).hasMessageContaining("INDEX:DESTROY");
-    assertThatThrownBy(() -> cacheServerMXBean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("CONTINUOUS_QUERY:EXECUTE");
-    assertThatThrownBy(() -> cacheServerMXBean.fetchLoadProbe()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
-    assertThatThrownBy(() -> cacheServerMXBean.getActiveCQCount()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
-    assertThatThrownBy(() -> cacheServerMXBean.stopContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("ONTINUOUS_QUERY:STOP");
-    assertThatThrownBy(() -> cacheServerMXBean.closeAllContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("ONTINUOUS_QUERY:STOP");
-    assertThatThrownBy(() -> cacheServerMXBean.isRunning()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
-    assertThatThrownBy(() -> cacheServerMXBean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.removeIndex("foo")).isInstanceOf(SecurityException.class).hasMessageContaining("INDEX:DESTROY");
+    assertThatThrownBy(() -> bean.executeContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("CONTINUOUS_QUERY:EXECUTE");
+    assertThatThrownBy(() -> bean.fetchLoadProbe()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.getActiveCQCount()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.stopContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("ONTINUOUS_QUERY:STOP");
+    assertThatThrownBy(() -> bean.closeAllContinuousQuery("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("ONTINUOUS_QUERY:STOP");
+    assertThatThrownBy(() -> bean.isRunning()).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.showClientQueueDetails("bar")).isInstanceOf(SecurityException.class).hasMessageContaining("JMX:GET");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DiskStoreMXBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DiskStoreMXBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DiskStoreMXBeanSecurityJUnitTest.java
new file mode 100644
index 0000000..df95287
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DiskStoreMXBeanSecurityJUnitTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.DiskStoreMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+@Category(IntegrationTest.class)
+public class DiskStoreMXBeanSecurityJUnitTest {
+  private static int jmxManagerPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+
+  private DiskStoreMXBean bean;
+
+  @ClassRule
+  public static JsonAuthorizationCacheStartRule serverRule = new JsonAuthorizationCacheStartRule(
+      jmxManagerPort, "cacheServer.json");
+
+  @Rule
+  public MBeanServerConnectionRule connectionRule = new MBeanServerConnectionRule(jmxManagerPort);
+
+  @BeforeClass
+  public static void beforeClass(){
+    serverRule.getCache().createDiskStoreFactory().create("diskstore");
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    bean = connectionRule.getProxyMBean(DiskStoreMXBean.class);
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "superuser", password = "1234567")
+  public void testAllAccess() throws Exception {
+    bean.flush();
+    bean.forceCompaction();
+    bean.forceRoll();
+    bean.getCompactionThreshold();
+    bean.getDiskDirectories();
+    bean.getDiskReadsRate();
+    bean.isAutoCompact();
+    bean.isForceCompactionAllowed();
+    bean.setDiskUsageCriticalPercentage(0.5f);
+    bean.setDiskUsageWarningPercentage(0.5f);
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "stranger", password = "1234567")
+  public void testNoAccess() throws Exception {
+    assertThatThrownBy(() -> bean.flush()).isInstanceOf(SecurityException.class).hasMessageContaining("DISKSTORE:FLUSH");
+    assertThatThrownBy(() -> bean.forceCompaction()).hasMessageContaining("DISKSTORE:COMPACT");
+    assertThatThrownBy(() -> bean.forceRoll()).hasMessageContaining("DISKSTORE:ROLL");
+    assertThatThrownBy(() -> bean.getCompactionThreshold()).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.getDiskDirectories()).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.getDiskReadsRate()).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.isAutoCompact()).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.isForceCompactionAllowed()).hasMessageContaining("JMX:GET");
+    assertThatThrownBy(() -> bean.setDiskUsageCriticalPercentage(0.5f)).hasMessageContaining("DISKSTORE:ALTER");
+    assertThatThrownBy(() -> bean.setDiskUsageWarningPercentage(0.5f)).hasMessageContaining("DISKSTORE:ALTER");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
index c0e1a8b..e86a8e6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/LockServiceMBeanAuthorizationJUnitTest.java
@@ -17,7 +17,6 @@
 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.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.locks.DLockService;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -48,7 +47,7 @@ public class LockServiceMBeanAuthorizationJUnitTest {
 
   @BeforeClass
   public static void beforeClassSetUp() {
-    Cache cache = CacheFactory.getAnyInstance();
+    Cache cache = serverRule.getCache();
     DLockService.create("test-lock-service", (InternalDistributedSystem) cache.getDistributedSystem(), false, true, true);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d73606c8/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 648ffb0..2aead40 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
@@ -11,6 +11,9 @@
         "MEMBER:SHUTDOWN",
         "MEMBER:STATUS",
         "DISKSTORE:COMPACT",
+        "DISKSTORE:FLUSH",
+        "DISKSTORE:ROLL",
+        "DISKSTORE:ALTER",
         "MANAGER:CREATE",
         "REGION:CREATE",
         "REGION:REBALANCE",