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 2019/04/12 15:16:16 UTC

[geode] branch develop updated: GEODE-6629: Allow disk stores to be specified for region creation in V2 Management API (#3440)

This is an automated email from the ASF dual-hosted git repository.

jinmeiliao pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 684ab13  GEODE-6629: Allow disk stores to be specified for region creation in V2 Management API (#3440)
684ab13 is described below

commit 684ab1370ee37895963979200dd7e82f374f72c1
Author: Jens Deppe <jd...@pivotal.io>
AuthorDate: Fri Apr 12 08:15:54 2019 -0700

    GEODE-6629: Allow disk stores to be specified for region creation in V2 Management API (#3440)
    
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
    Co-authored-by: Jinmei Liao <ji...@pivotal.io>
---
 ...ateRegionWithDiskstoreAndSecurityDUnitTest.java | 122 +++++++++++++++++++++
 .../api/LocatorClusterManagementService.java       |   2 +-
 .../validators/RegionConfigValidator.java          |  15 +++
 .../validators/RegionConfigValidatorTest.java      |  35 +++++-
 .../cache/configuration/BasicRegionConfig.java     |  18 +--
 .../geode/cache/configuration/RegionType.java      |  20 +++-
 .../ClientClusterManagementServiceDUnitTest.java   |   2 +-
 7 files changed, 193 insertions(+), 21 deletions(-)

diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/client/CreateRegionWithDiskstoreAndSecurityDUnitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/client/CreateRegionWithDiskstoreAndSecurityDUnitTest.java
new file mode 100644
index 0000000..dba0586
--- /dev/null
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/client/CreateRegionWithDiskstoreAndSecurityDUnitTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.management.client;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.cache.configuration.BasicRegionConfig;
+import org.apache.geode.cache.configuration.RegionAttributesType;
+import org.apache.geode.cache.configuration.RegionType;
+import org.apache.geode.examples.SimpleSecurityManager;
+import org.apache.geode.management.api.ClusterManagementResult;
+import org.apache.geode.management.api.ClusterManagementService;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+
+public class CreateRegionWithDiskstoreAndSecurityDUnitTest {
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule(1);
+
+  @Rule
+  public GfshCommandRule gfsh = new GfshCommandRule();
+
+  private MemberVM server;
+  private MemberVM locator;
+
+  @Before
+  public void before() throws Exception {
+    locator = cluster.startLocatorVM(0,
+        c -> c.withHttpService().withSecurityManager(SimpleSecurityManager.class));
+    int locatorPort = locator.getPort();
+    server = cluster.startServerVM(1,
+        s -> s.withConnectionToLocator(locatorPort)
+            .withCredential("cluster", "cluster"));
+
+    gfsh.secureConnectAndVerify(locator.getPort(), GfshCommandRule.PortType.locator,
+        "data,cluster", "data,cluster");
+  }
+
+  @Test
+  public void createReplicateRegionWithDiskstoreWithoutDataManage() throws Exception {
+    gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
+        cluster.getWorkingDirRoot())).statusIsSuccess();
+
+    BasicRegionConfig regionConfig = new BasicRegionConfig();
+    regionConfig.setName("REGION1");
+    regionConfig.setType(RegionType.REPLICATE_PERSISTENT);
+
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDiskStoreName("DISKSTORE");
+    regionConfig.setRegionAttributes(attributes);
+
+    ClusterManagementService client =
+        ClusterManagementServiceProvider.getService("localhost", locator.getHttpPort(), null, null,
+            "user", "user");
+    ClusterManagementResult result = client.create(regionConfig);
+    assertThat(result.isSuccessful()).isFalse();
+    assertThat(result.getStatusCode()).isEqualTo(ClusterManagementResult.StatusCode.UNAUTHORIZED);
+    assertThat(result.getStatusMessage()).isEqualTo("user not authorized for DATA:MANAGE");
+  }
+
+  @Test
+  public void createReplicateRegionWithDiskstoreWithoutClusterWrite() throws Exception {
+    gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
+        cluster.getWorkingDirRoot())).statusIsSuccess();
+
+    BasicRegionConfig regionConfig = new BasicRegionConfig();
+    regionConfig.setName("REGION1");
+    regionConfig.setType(RegionType.REPLICATE_PERSISTENT);
+
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDiskStoreName("DISKSTORE");
+    regionConfig.setRegionAttributes(attributes);
+
+    ClusterManagementService client =
+        ClusterManagementServiceProvider.getService("localhost", locator.getHttpPort(), null, null,
+            "data", "data");
+    ClusterManagementResult result = client.create(regionConfig);
+    assertThat(result.isSuccessful()).isFalse();
+    assertThat(result.getStatusCode()).isEqualTo(ClusterManagementResult.StatusCode.UNAUTHORIZED);
+    assertThat(result.getStatusMessage()).isEqualTo("data not authorized for CLUSTER:WRITE:DISK");
+  }
+
+  @Test
+  public void createReplicateRegionWithDiskstoreSuccess() throws Exception {
+    gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
+        cluster.getWorkingDirRoot())).statusIsSuccess();
+
+    BasicRegionConfig regionConfig = new BasicRegionConfig();
+    regionConfig.setName("REGION1");
+    regionConfig.setType(RegionType.REPLICATE_PERSISTENT);
+
+    RegionAttributesType attributes = new RegionAttributesType();
+    attributes.setDiskStoreName("DISKSTORE");
+    regionConfig.setRegionAttributes(attributes);
+
+    ClusterManagementService client =
+        ClusterManagementServiceProvider.getService("localhost", locator.getHttpPort(), null, null,
+            "data,cluster", "data,cluster");
+    ClusterManagementResult result = client.create(regionConfig);
+    assertThat(result.isSuccessful()).isTrue();
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
index d64b06c..9051b34 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/api/LocatorClusterManagementService.java
@@ -65,7 +65,7 @@ public class LocatorClusterManagementService implements ClusterManagementService
     managers.put(MemberConfig.class, new MemberConfigManager(cache));
 
     // initialize the list of validators
-    validators.put(BasicRegionConfig.class, new RegionConfigValidator());
+    validators.put(BasicRegionConfig.class, new RegionConfigValidator(cache));
   }
 
   @VisibleForTesting
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
index 40b7e03..ca941fb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidator.java
@@ -19,9 +19,16 @@ import org.apache.geode.cache.configuration.BasicRegionConfig;
 import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.CacheElement;
 import org.apache.geode.cache.configuration.RegionType;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.RegionNameValidation;
+import org.apache.geode.security.ResourcePermission;
 
 public class RegionConfigValidator implements ConfigurationValidator<BasicRegionConfig> {
+  private InternalCache cache;
+
+  public RegionConfigValidator(InternalCache cache) {
+    this.cache = cache;
+  }
 
   @Override
   public void validate(BasicRegionConfig config)
@@ -41,10 +48,18 @@ public class RegionConfigValidator implements ConfigurationValidator<BasicRegion
       // by management v2 api.
       try {
         RegionType.valueOf(type);
+
       } catch (IllegalArgumentException e) {
         throw new IllegalArgumentException(
             String.format("Type %s is not supported in Management V2 API.", type));
       }
+
+      // additional authorization
+      if (config.getRegionAttributes().getDataPolicy().isPersistent()) {
+        cache.getSecurityService()
+            .authorize(ResourcePermission.Resource.CLUSTER, ResourcePermission.Operation.WRITE,
+                ResourcePermission.Target.DISK);
+      }
     }
 
   }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
index e235653..df44a5e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/validators/RegionConfigValidatorTest.java
@@ -19,29 +19,54 @@ package org.apache.geode.management.internal.configuration.validators;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.configuration.RegionType;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.security.ResourcePermission;
 
 public class RegionConfigValidatorTest {
 
   private RegionConfigValidator validator;
   private RegionConfig config;
+  private SecurityService securityService;
 
   @Before
   public void before() throws Exception {
-    validator = new RegionConfigValidator();
+    InternalCache cache = mock(InternalCache.class);
+    securityService = mock(SecurityService.class);
+    when(cache.getSecurityService()).thenReturn(securityService);
+    validator = new RegionConfigValidator(cache);
     config = new RegionConfig();
   }
 
   @Test
+  public void checkSecurityForDiskAccess() {
+    config.setName("regionName");
+    config.setType(RegionType.REPLICATE_PERSISTENT);
+    validator.validate(config);
+
+    verify(securityService).authorize(ResourcePermission.Resource.CLUSTER,
+        ResourcePermission.Operation.WRITE, ResourcePermission.Target.DISK);
+    assertThat(config.getType()).isEqualTo("REPLICATE_PERSISTENT");
+  }
+
+  @Test
   public void noChangesWhenTypeIsSet() {
     config.setName("regionName");
     config.setType(RegionType.REPLICATE);
     validator.validate(config);
+
+    verify(securityService, times(0)).authorize(any());
     assertThat(config.getType()).isEqualTo("REPLICATE");
   }
 
@@ -58,6 +83,8 @@ public class RegionConfigValidatorTest {
   public void defaultsTypeToPartitioned() {
     config.setName("regionName");
     validator.validate(config);
+
+    verify(securityService, times(0)).authorize(any());
     assertThat(config.getType()).isEqualTo("PARTITION");
   }
 
@@ -66,6 +93,8 @@ public class RegionConfigValidatorTest {
     assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
         IllegalArgumentException.class)
         .hasMessageContaining("Name of the region has to be specified");
+
+    verify(securityService, times(0)).authorize(any());
   }
 
   @Test
@@ -74,6 +103,8 @@ public class RegionConfigValidatorTest {
     assertThatThrownBy(() -> validator.validate(config)).isInstanceOf(
         IllegalArgumentException.class)
         .hasMessageContaining("Region names may not begin with a double-underscore");
+
+    verify(securityService, times(0)).authorize(any());
   }
 
   @Test
@@ -83,5 +114,7 @@ public class RegionConfigValidatorTest {
         IllegalArgumentException.class)
         .hasMessageContaining(
             "Region names may only be alphanumeric and may contain hyphens or underscores");
+
+    verify(securityService, times(0)).authorize(any());
   }
 }
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/BasicRegionConfig.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/BasicRegionConfig.java
index e4572ad..04c1b18 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/BasicRegionConfig.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/BasicRegionConfig.java
@@ -23,7 +23,6 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlType;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
-import io.swagger.annotations.ApiModelProperty;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.management.api.RestfulEndpoint;
@@ -48,7 +47,6 @@ public class BasicRegionConfig extends CacheElement implements RestfulEndpoint {
 
   public static final String REGION_CONFIG_ENDPOINT = "/regions";
 
-  @ApiModelProperty(hidden = true)
   @XmlElement(name = "region-attributes", namespace = "http://geode.apache.org/schema/cache")
   protected RegionAttributesType regionAttributes;
 
@@ -81,9 +79,7 @@ public class BasicRegionConfig extends CacheElement implements RestfulEndpoint {
   /**
    * Gets the value of the name property.
    *
-   * possible object is
-   * {@link String }
-   *
+   * possible object is {@link String }
    */
   public String getName() {
     return name;
@@ -92,9 +88,7 @@ public class BasicRegionConfig extends CacheElement implements RestfulEndpoint {
   /**
    * Sets the value of the name property.
    *
-   * allowed object is
-   * {@link String }
-   *
+   * allowed object is {@link String }
    */
   public void setName(String value) throws IllegalArgumentException {
     if (value == null) {
@@ -116,9 +110,7 @@ public class BasicRegionConfig extends CacheElement implements RestfulEndpoint {
   /**
    * Gets the value of the type property.
    *
-   * possible object is
-   * {@link String }
-   *
+   * possible object is {@link String }
    */
   public String getType() {
     return type;
@@ -127,9 +119,7 @@ public class BasicRegionConfig extends CacheElement implements RestfulEndpoint {
   /**
    * Sets the value of the type property.
    *
-   * allowed object is
-   * {@link String }
-   *
+   * allowed object is {@link String }
    */
   public void setType(RegionType regionType) {
     if (regionType != null) {
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionType.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionType.java
index f192403..dade0b6 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionType.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionType.java
@@ -24,12 +24,24 @@ import org.apache.geode.annotations.Experimental;
 @Experimental
 public enum RegionType {
   PARTITION,
+  PARTITION_REDUNDANT,
+  PARTITION_PERSISTENT,
+  PARTITION_REDUNDANT_PERSISTENT,
+  PARTITION_OVERFLOW,
+  PARTITION_REDUNDANT_OVERFLOW,
+  PARTITION_PERSISTENT_OVERFLOW,
+  PARTITION_REDUNDANT_PERSISTENT_OVERFLOW,
+  PARTITION_HEAP_LRU,
+  PARTITION_REDUNDANT_HEAP_LRU,
+
   PARTITION_PROXY,
   PARTITION_PROXY_REDUNDANT,
-  PARTITION_HEAP_LRU,
-  PARTITION_OVERFLOW,
+
   REPLICATE,
-  REPLICATE_PROXY,
+  REPLICATE_PERSISTENT,
+  REPLICATE_OVERFLOW,
+  REPLICATE_PERSISTENT_OVERFLOW,
   REPLICATE_HEAP_LRU,
-  REPLICATE_OVERFLOW
+
+  REPLICATE_PROXY
 }
diff --git a/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/ClientClusterManagementServiceDUnitTest.java b/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/ClientClusterManagementServiceDUnitTest.java
index 3e1e9d2..16914c4 100644
--- a/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/ClientClusterManagementServiceDUnitTest.java
+++ b/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/ClientClusterManagementServiceDUnitTest.java
@@ -67,7 +67,7 @@ public class ClientClusterManagementServiceDUnitTest {
   public void createRegion() {
     BasicRegionConfig region = new BasicRegionConfig();
     region.setName("customer");
-    region.setType(RegionType.REPLICATE);
+    region.setType(RegionType.REPLICATE_PERSISTENT);
 
     ClusterManagementResult result = client.create(region);