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/05/09 15:14:32 UTC

[geode] branch develop updated: GEODE-6744: list index rest api (#3550)

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 7e59100  GEODE-6744: list index rest api (#3550)
7e59100 is described below

commit 7e591000fd9a17b36ee2e22649f6b39cab226910
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Thu May 9 08:14:17 2019 -0700

    GEODE-6744: list index rest api (#3550)
    
    Co-authored-by: Owen Nichols <on...@pivotal.io>
---
 .../rest/ListIndexManagementDUnitTest.java         | 181 +++++++++++++++++++++
 .../integrationTest/resources/assembly_content.txt |   1 +
 .../api/LocatorClusterManagementService.java       |  17 ++
 .../geode/cache/configuration/CacheConfigTest.java |   2 +-
 .../cache/configuration/RegionConfigTest.java      |  85 ++++++++++
 .../geode/cache/configuration/RegionConfig.java    |  56 ++++++-
 .../management/api/ClusterManagementResult.java    |   4 +-
 .../management/api/ClusterManagementService.java   |   2 +
 ...{RuntimeRegionConfig.java => RuntimeIndex.java} |  26 +--
 .../configuration/RuntimeRegionConfig.java         |  18 ++
 .../internal/ClientClusterManagementService.java   |  13 +-
 .../configuration/CacheElementJsonMappingTest.java |  23 +++
 .../client/MemberManagementServiceDUnitTest.java   |   2 +-
 .../controllers/MemberManagementController.java    |   8 +-
 .../controllers/RegionManagementController.java    |  64 +++++++-
 15 files changed, 463 insertions(+), 39 deletions(-)

diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListIndexManagementDUnitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListIndexManagementDUnitTest.java
new file mode 100644
index 0000000..86ad530
--- /dev/null
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/management/internal/rest/ListIndexManagementDUnitTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.internal.rest;
+
+import static org.apache.geode.test.junit.assertions.ClusterManagementResultAssert.assertManagementResult;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.cache.configuration.RegionConfig;
+import org.apache.geode.cache.configuration.RegionType;
+import org.apache.geode.management.api.ClusterManagementResult;
+import org.apache.geode.management.api.ClusterManagementService;
+import org.apache.geode.management.client.JavaClientClusterManagementServiceConfig;
+import org.apache.geode.management.configuration.RuntimeIndex;
+import org.apache.geode.management.configuration.RuntimeRegionConfig;
+import org.apache.geode.management.internal.ClientClusterManagementService;
+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 ListIndexManagementDUnitTest {
+  private static MemberVM locator, server;
+
+  private RegionConfig regionConfig;
+
+  @ClassRule
+  public static ClusterStartupRule lsRule = new ClusterStartupRule();
+
+  @ClassRule
+  public static GfshCommandRule gfsh = new GfshCommandRule();
+
+  private static ClusterManagementService cms;
+
+  @BeforeClass
+  public static void beforeclass() throws Exception {
+    locator = lsRule.startLocatorVM(0, l -> l.withHttpService());
+    server = lsRule.startServerVM(1, locator.getPort());
+
+    cms = new ClientClusterManagementService(
+        JavaClientClusterManagementServiceConfig.builder().setHost("localhost")
+            .setPort(locator.getHttpPort()).build());
+
+    RegionConfig config = new RegionConfig();
+    config.setName("region1");
+    config.setType(RegionType.REPLICATE);
+    cms.create(config);
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/region1", 1);
+
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndAssertThat(
+        "create index --name=index1 --type=key --expression=id --region=/region1")
+        .statusIsSuccess();
+    gfsh.executeAndAssertThat(
+        "create index --name=index2 --type=key --expression=key --region=/region1")
+        .statusIsSuccess();
+  }
+
+  @Before
+  public void before() throws Exception {
+    regionConfig = new RegionConfig();
+  }
+
+  @Test
+  public void listRegion() {
+    List<RuntimeRegionConfig> result =
+        cms.list(new RegionConfig()).getResult(RuntimeRegionConfig.class);
+    assertThat(result).hasSize(1);
+  }
+
+  @Test
+  public void getRegion() throws Exception {
+    regionConfig.setName("region1");
+    List<RuntimeRegionConfig> regions = cms.get(regionConfig).getResult(RuntimeRegionConfig.class);
+    assertThat(regions).hasSize(1);
+    RuntimeRegionConfig region = regions.get(0);
+    List<RegionConfig.Index> indexes = region.getIndexes();
+    assertThat(indexes).hasSize(2);
+  }
+
+  @Test
+  public void getNonExistRegion() throws Exception {
+    regionConfig.setName("notExist");
+    assertManagementResult(cms.get(regionConfig)).failed().hasStatusCode(
+        ClusterManagementResult.StatusCode.ENTITY_NOT_FOUND);
+  }
+
+  @Test
+  public void listIndexForOneRegion() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    ClusterManagementResult list = cms.list(index);
+    List<RuntimeIndex> result = list.getResult(RuntimeIndex.class);
+    assertThat(result).hasSize(2);
+  }
+
+  @Test
+  public void getIndex() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    index.setName("index1");
+    ClusterManagementResult list = cms.get(index);
+    List<RuntimeIndex> result = list.getResult(RuntimeIndex.class);
+    assertThat(result).hasSize(1);
+    RuntimeIndex runtimeIndex = result.get(0);
+    assertThat(runtimeIndex.getRegionName()).isEqualTo("region1");
+    assertThat(runtimeIndex.getName()).isEqualTo("index1");
+    assertThat(runtimeIndex.getFromClause()).isEqualTo("/region1");
+    assertThat(runtimeIndex.getExpression()).isEqualTo("id");
+  }
+
+  @Test
+  public void getIndexWithoutIndexId() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    assertThatThrownBy(() -> cms.get(index)).isInstanceOf(IllegalArgumentException.class)
+        .hasMessageContaining("Id is required");
+  }
+
+  @Test
+  public void getIndexWithoutRegionName() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setName("index1");
+    assertThatThrownBy(() -> cms.get(index)).isInstanceOf(IllegalArgumentException.class)
+        .hasMessageContaining("regionName is required");
+  }
+
+  @Test
+  public void listIndexesWithIdFilter() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    index.setName("index1");
+    ClusterManagementResult list = cms.list(index);
+    List<RuntimeIndex> result = list.getResult(RuntimeIndex.class);
+    assertThat(result).hasSize(1);
+    RuntimeIndex runtimeIndex = result.get(0);
+    assertThat(runtimeIndex.getRegionName()).isEqualTo("region1");
+    assertThat(runtimeIndex.getName()).isEqualTo("index1");
+    assertThat(runtimeIndex.getFromClause()).isEqualTo("/region1");
+    assertThat(runtimeIndex.getExpression()).isEqualTo("id");
+  }
+
+  @Test
+  public void getNonExistingIndex() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    index.setName("index333");
+    assertManagementResult(cms.get(index)).failed()
+        .hasStatusCode(ClusterManagementResult.StatusCode.ENTITY_NOT_FOUND);
+  }
+
+  @Test
+  public void listNonExistingIndexesWithIdFilter() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setRegionName("region1");
+    index.setName("index333");
+    ClusterManagementResult list = cms.list(index);
+    List<RuntimeIndex> result = list.getResult(RuntimeIndex.class);
+    assertThat(result).hasSize(0);
+    assertThat(list.isSuccessful()).isTrue();
+  }
+}
diff --git a/geode-assembly/src/integrationTest/resources/assembly_content.txt b/geode-assembly/src/integrationTest/resources/assembly_content.txt
index a6a30ae..b0c3774 100644
--- a/geode-assembly/src/integrationTest/resources/assembly_content.txt
+++ b/geode-assembly/src/integrationTest/resources/assembly_content.txt
@@ -718,6 +718,7 @@ javadoc/org/apache/geode/management/client/package-tree.html
 javadoc/org/apache/geode/management/configuration/MemberConfig.CacheServerConfig.html
 javadoc/org/apache/geode/management/configuration/MemberConfig.html
 javadoc/org/apache/geode/management/configuration/RuntimeCacheElement.html
+javadoc/org/apache/geode/management/configuration/RuntimeIndex.html
 javadoc/org/apache/geode/management/configuration/RuntimeRegionConfig.html
 javadoc/org/apache/geode/management/configuration/package-frame.html
 javadoc/org/apache/geode/management/configuration/package-summary.html
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 ffa9342..86d52b2 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
@@ -54,6 +54,7 @@ import org.apache.geode.management.internal.configuration.validators.CacheElemen
 import org.apache.geode.management.internal.configuration.validators.ConfigurationValidator;
 import org.apache.geode.management.internal.configuration.validators.RegionConfigValidator;
 import org.apache.geode.management.internal.exceptions.EntityExistsException;
+import org.apache.geode.management.internal.exceptions.EntityNotFoundException;
 
 public class LocatorClusterManagementService implements ClusterManagementService {
   private static final Logger logger = LogService.getLogger();
@@ -220,6 +221,22 @@ public class LocatorClusterManagementService implements ClusterManagementService
     return result;
   }
 
+  @Override
+  public ClusterManagementResult get(CacheElement config) {
+    ClusterManagementResult list = list(config);
+    List<RuntimeCacheElement> result = list.getResult(RuntimeCacheElement.class);
+    if (result.size() == 0) {
+      throw new EntityNotFoundException(
+          config.getClass().getSimpleName() + " with id = " + config.getId() + " not found.");
+    }
+
+    if (result.size() > 1) {
+      throw new IllegalStateException(
+          "Expect only one matching " + config.getClass().getSimpleName());
+    }
+    return list;
+  }
+
 
   @Override
   public boolean isConnected() {
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
index f81cdd2..87347e8 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/CacheConfigTest.java
@@ -81,7 +81,7 @@ public class CacheConfigTest {
     assertThat(index.isKeyIndex()).isTrue();
     assertThat(index.getName()).isEqualTo("indexName");
     assertThat(index.getExpression()).isEqualTo("expression");
-    assertThat(index.getType()).isEqualTo("range");
+    assertThat(index.getType()).isEqualTo("key");
   }
 
 
diff --git a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
index c04739f..82d6661 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/configuration/RegionConfigTest.java
@@ -91,4 +91,89 @@ public class RegionConfigTest {
     String xml = service.marshall(cacheConfig);
     assertThat(xml).contains("<region name=\"test\" refid=\"REPLICATE\"");
   }
+
+  @Test
+  public void indexType() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    assertThat(index.isKeyIndex()).isNull();
+    assertThat(index.getType()).isEqualTo("range");
+
+    index.setKeyIndex(true);
+    assertThat(index.isKeyIndex()).isTrue();
+    assertThat(index.getType()).isEqualTo("key");
+
+    index.setKeyIndex(false);
+    assertThat(index.isKeyIndex()).isFalse();
+    assertThat(index.getType()).isEqualTo("range");
+
+    index.setType("hash");
+    assertThat(index.isKeyIndex()).isFalse();
+    assertThat(index.getType()).isEqualTo("hash");
+
+    index.setType("key");
+    assertThat(index.isKeyIndex()).isTrue();
+    assertThat(index.getType()).isEqualTo("key");
+  }
+
+  @Test
+  public void index() throws Exception {
+    String xml = "<region name=\"region1\" refid=\"REPLICATE\">\n"
+        + "<region-attributes data-policy=\"replicate\" scope=\"distributed-ack\" concurrency-checks-enabled=\"true\"/>\n"
+        + "<index name=\"index1\" expression=\"id\" from-clause=\"/region1\" key-index=\"true\"/>\n"
+        + "</region>";
+
+    RegionConfig regionConfig = service.unMarshall(xml, RegionConfig.class);
+
+    RegionConfig.Index index = regionConfig.getIndexes().get(0);
+    assertThat(index.isKeyIndex()).isTrue();
+    assertThat(index.getType()).isEqualTo("key");
+
+    String json = GeodeJsonMapper.getMapper().writeValueAsString(index);
+    System.out.println(json);
+    RegionConfig.Index newIndex =
+        GeodeJsonMapper.getMapper().readValue(json, RegionConfig.Index.class);
+    assertThat(newIndex.isKeyIndex()).isTrue();
+    assertThat(newIndex.getType()).isEqualTo("key");
+
+    CacheConfig cacheConfig = new CacheConfig();
+    regionConfig.getIndexes().clear();
+    regionConfig.getIndexes().add(newIndex);
+    cacheConfig.getRegions().add(regionConfig);
+
+    // the end xml should not have "type" attribute in index definition
+    String newXml = service.marshall(cacheConfig);
+    System.out.println(newXml);
+    assertThat(newXml).doesNotContain("type=");
+  }
+
+  @Test
+  public void invalidRegionName() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setExpression("id");
+    index.setName("index1");
+    index.setFromClause("/regionA");
+
+    assertThatThrownBy(() -> index.setRegionName("regionB"))
+        .isInstanceOf(IllegalArgumentException.class);
+  }
+
+  @Test
+  public void fromClauseInferredByRegionName() throws Exception {
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setExpression("id");
+    index.setName("index1");
+    index.setRegionName("regionA");
+
+    assertThat(index.getFromClause()).isEqualTo("/regionA");
+
+    CacheConfig cacheConfig = new CacheConfig();
+    regionConfig.getIndexes().clear();
+    regionConfig.getIndexes().add(index);
+    cacheConfig.getRegions().add(regionConfig);
+
+    // the end xml should not have "type" attribute in index definition
+    String newXml = service.marshall(cacheConfig);
+    System.out.println(newXml);
+    assertThat(newXml).contains("from-clause=");
+  }
 }
diff --git a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
index 4b83c2f..9a0bb88 100644
--- a/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
+++ b/geode-management/src/main/java/org/apache/geode/cache/configuration/RegionConfig.java
@@ -28,9 +28,11 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAnyElement;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.commons.lang3.StringUtils;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.management.api.RestfulEndpoint;
@@ -658,7 +660,7 @@ public class RegionConfig extends CacheElement implements RestfulEndpoint {
    *
    */
   @XmlAccessorType(XmlAccessType.FIELD)
-  public static class Index extends CacheElement {
+  public static class Index extends CacheElement implements RestfulEndpoint {
     @XmlAttribute(name = "name", required = true)
     protected String name;
     @XmlAttribute(name = "expression")
@@ -671,6 +673,19 @@ public class RegionConfig extends CacheElement implements RestfulEndpoint {
     protected Boolean keyIndex;
     @XmlAttribute(name = "type")
     protected String type; // for non-key index type, range or hash
+    @XmlTransient
+    protected String regionName;
+
+    public Index() {}
+
+    public Index(Index index) {
+      this.name = index.name;
+      this.expression = index.expression;
+      this.fromClause = index.fromClause;
+      this.imports = index.imports;
+      this.keyIndex = index.keyIndex;
+      this.type = index.type;
+    }
 
     /**
      * Gets the value of the name property.
@@ -790,6 +805,12 @@ public class RegionConfig extends CacheElement implements RestfulEndpoint {
      *
      */
     public String getType() {
+      // this should return a "key" value because some production code relies on this method
+      // returning a type string that would turn into IndexType enum object
+      if (keyIndex == Boolean.TRUE) {
+        return "key";
+      }
+
       if (type == null) {
         return "range";
       } else {
@@ -808,20 +829,47 @@ public class RegionConfig extends CacheElement implements RestfulEndpoint {
      *             {@link #setKeyIndex(Boolean)}
      */
     public void setType(String value) {
-      if ("range".equalsIgnoreCase(value) || "hash".equalsIgnoreCase(value)
-          || "key".equalsIgnoreCase(value)) {
+      if ("range".equalsIgnoreCase(value) || "hash".equalsIgnoreCase(value)) {
         this.type = value.toLowerCase();
+        setKeyIndex(false);
+      }
+      // we need to avoid setting the "type" to key since by xsd definition, it should only contain
+      // "hash" and "range" value.
+      else if ("key".equalsIgnoreCase(value)) {
+        this.type = null;
+        setKeyIndex(true);
       } else {
         throw new IllegalArgumentException("Invalid index type " + value);
       }
+    }
+
+    public String getRegionName() {
+      return regionName;
+    }
 
-      setKeyIndex("key".equalsIgnoreCase(value));
+    public void setRegionName(String regionName) {
+      this.regionName = regionName;
+      if (fromClause == null) {
+        fromClause = "/" + regionName;
+      } else if (!fromClause.contains(regionName)) {
+        throw new IllegalArgumentException(
+            "Invalid regionName for this index with fromClause = " + fromClause);
+      }
     }
 
     @Override
+    @JsonIgnore
     public String getId() {
       return getName();
     }
+
+    @Override
+    public String getEndpoint() {
+      if (StringUtils.isBlank(regionName)) {
+        throw new IllegalArgumentException("regionName is required.");
+      }
+      return RegionConfig.REGION_CONFIG_ENDPOINT + "/" + regionName + "/indexes";
+    }
   }
 
 }
diff --git a/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementResult.java b/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementResult.java
index aba3641..9bc0848 100644
--- a/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementResult.java
+++ b/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementResult.java
@@ -62,7 +62,7 @@ public class ClusterManagementResult {
   private String statusMessage;
 
   @JsonProperty
-  private List<RuntimeCacheElement> result = new ArrayList<>();
+  private List<? extends RuntimeCacheElement> result = new ArrayList<>();
 
   public ClusterManagementResult() {}
 
@@ -116,7 +116,7 @@ public class ClusterManagementResult {
     return result.stream().map(clazz::cast).collect(Collectors.toList());
   }
 
-  public void setResult(List<RuntimeCacheElement> result) {
+  public void setResult(List<? extends RuntimeCacheElement> result) {
     this.result = result;
   }
 }
diff --git a/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementService.java b/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementService.java
index cc36dde..e45d656 100644
--- a/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementService.java
+++ b/geode-management/src/main/java/org/apache/geode/management/api/ClusterManagementService.java
@@ -62,6 +62,8 @@ public interface ClusterManagementService {
 
   ClusterManagementResult list(CacheElement config);
 
+  ClusterManagementResult get(CacheElement config);
+
   /**
    * Test to see if this instance of ClusterManagementService retrieved from the client side is
    * properly connected to the locator or not
diff --git a/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java b/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeIndex.java
similarity index 69%
copy from geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java
copy to geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeIndex.java
index da4367e..3c3511c 100644
--- a/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java
+++ b/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeIndex.java
@@ -15,31 +15,19 @@
 
 package org.apache.geode.management.configuration;
 
-
 import java.util.List;
 
-import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.configuration.RegionConfig;
 
-@Experimental
-public class RuntimeRegionConfig extends RegionConfig implements RuntimeCacheElement {
-  private long entryCount;
-
-  public RuntimeRegionConfig() {}
-
-  public RuntimeRegionConfig(RegionConfig config) {
-    super(config);
-  }
-
-  public long getEntryCount() {
-    return entryCount;
+public class RuntimeIndex extends RegionConfig.Index implements RuntimeCacheElement {
+  @Override
+  public List<String> getGroups() {
+    return groups;
   }
 
-  public void setEntryCount(long entrySize) {
-    this.entryCount = entrySize;
-  }
+  public RuntimeIndex() {};
 
-  public List<String> getGroups() {
-    return groups;
+  public RuntimeIndex(RegionConfig.Index index) {
+    super(index);
   }
 }
diff --git a/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java b/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java
index da4367e..683ecb9 100644
--- a/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java
+++ b/geode-management/src/main/java/org/apache/geode/management/configuration/RuntimeRegionConfig.java
@@ -17,6 +17,10 @@ package org.apache.geode.management.configuration;
 
 
 import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.commons.lang3.StringUtils;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.configuration.RegionConfig;
@@ -42,4 +46,18 @@ public class RuntimeRegionConfig extends RegionConfig implements RuntimeCacheEle
   public List<String> getGroups() {
     return groups;
   }
+
+  public List<RuntimeIndex> getRuntimeIndexes(String indexId) {
+    Stream<Index> stream = getIndexes().stream();
+    if (StringUtils.isNotBlank(indexId)) {
+      stream = stream.filter(i -> i.getId().equals(indexId));
+    }
+    return stream
+        .map(e -> {
+          RuntimeIndex index = new RuntimeIndex(e);
+          index.setRegionName(getName());
+          return index;
+        })
+        .collect(Collectors.toList());
+  }
 }
diff --git a/geode-management/src/main/java/org/apache/geode/management/internal/ClientClusterManagementService.java b/geode-management/src/main/java/org/apache/geode/management/internal/ClientClusterManagementService.java
index a2289bf..05c0ec2 100644
--- a/geode-management/src/main/java/org/apache/geode/management/internal/ClientClusterManagementService.java
+++ b/geode-management/src/main/java/org/apache/geode/management/internal/ClientClusterManagementService.java
@@ -18,6 +18,7 @@ package org.apache.geode.management.internal;
 
 
 import org.apache.commons.lang3.NotImplementedException;
+import org.apache.commons.lang3.StringUtils;
 import org.springframework.web.client.RestTemplate;
 
 import org.apache.geode.cache.configuration.CacheElement;
@@ -80,6 +81,17 @@ public class ClientClusterManagementService implements ClusterManagementService
         .getBody();
   }
 
+  @Override
+  public ClusterManagementResult get(CacheElement config) {
+    if (StringUtils.isBlank(config.getId())) {
+      throw new IllegalArgumentException("Id is required.");
+    }
+    String endPoint = getEndpoint(config);
+    return serviceConfig.getRestTemplate()
+        .getForEntity(VERSION + endPoint + "/{id}", ClusterManagementResult.class, config.getId())
+        .getBody();
+  }
+
   public RestTemplate getRestTemplate() {
     return serviceConfig.getRestTemplate();
   }
@@ -90,7 +102,6 @@ public class ClientClusterManagementService implements ClusterManagementService
           String.format("The config type %s does not have a RESTful endpoint defined",
               config.getClass().getName()));
     }
-
     return ((RestfulEndpoint) config).getEndpoint();
   }
 
diff --git a/geode-management/src/test/java/org/apache/geode/cache/configuration/CacheElementJsonMappingTest.java b/geode-management/src/test/java/org/apache/geode/cache/configuration/CacheElementJsonMappingTest.java
index 3ed245d..dee25bb 100644
--- a/geode-management/src/test/java/org/apache/geode/cache/configuration/CacheElementJsonMappingTest.java
+++ b/geode-management/src/test/java/org/apache/geode/cache/configuration/CacheElementJsonMappingTest.java
@@ -27,6 +27,7 @@ import org.junit.Test;
 import org.apache.geode.management.api.ClusterManagementResult;
 import org.apache.geode.management.configuration.MemberConfig;
 import org.apache.geode.management.configuration.RuntimeCacheElement;
+import org.apache.geode.management.configuration.RuntimeIndex;
 import org.apache.geode.management.configuration.RuntimeRegionConfig;
 import org.apache.geode.util.internal.GeodeJsonMapper;
 
@@ -150,6 +151,28 @@ public class CacheElementJsonMappingTest {
   }
 
   @Test
+  public void serializeRuntimeRegionConfigWithIndex() throws Exception {
+    RegionConfig config = new RegionConfig();
+    config.setName("region1");
+    config.setType(RegionType.REPLICATE);
+    config.setGroup("group1");
+    RegionConfig.Index index = new RegionConfig.Index();
+    index.setName("index1");
+    index.setFromClause("/region1 r");
+    index.setExpression("id");
+    config.getIndexes().add(index);
+    RuntimeRegionConfig runtimeConfig = new RuntimeRegionConfig(config);
+    String json = mapper.writeValueAsString(runtimeConfig);
+    System.out.println(json);
+
+    runtimeConfig = mapper.readValue(json, RuntimeRegionConfig.class);
+    assertThat(runtimeConfig.getGroups()).containsExactly("group1");
+    List<RuntimeIndex> runtimeIndexes = runtimeConfig.getRuntimeIndexes(null);
+    assertThat(runtimeIndexes).hasSize(1);
+    assertThat(runtimeIndexes.get(0).getRegionName()).isEqualTo("region1");
+  }
+
+  @Test
   public void serializeGroupCluster() throws Exception {
     RegionConfig config = new RegionConfig();
     config.setName("test");
diff --git a/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/MemberManagementServiceDUnitTest.java b/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/MemberManagementServiceDUnitTest.java
index 08450a3..c2529ea 100644
--- a/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/MemberManagementServiceDUnitTest.java
+++ b/geode-web-management/src/distributedTest/java/org/apache/geode/management/client/MemberManagementServiceDUnitTest.java
@@ -164,6 +164,6 @@ public class MemberManagementServiceDUnitTest {
         .andExpect(status().isNotFound())
         .andExpect(jsonPath("$.statusCode", is("ENTITY_NOT_FOUND")))
         .andExpect(jsonPath("$.statusMessage",
-            is("Unable to find the member with id = server")));
+            is("MemberConfig with id = server not found.")));
   }
 }
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/MemberManagementController.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/MemberManagementController.java
index 1da14da..a2affbb 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/MemberManagementController.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/MemberManagementController.java
@@ -30,8 +30,6 @@ import org.springframework.web.bind.annotation.RequestParam;
 
 import org.apache.geode.management.api.ClusterManagementResult;
 import org.apache.geode.management.configuration.MemberConfig;
-import org.apache.geode.management.configuration.RuntimeCacheElement;
-import org.apache.geode.management.internal.exceptions.EntityNotFoundException;
 
 @Controller("members")
 @RequestMapping(MANAGEMENT_API_VERSION)
@@ -42,11 +40,7 @@ public class MemberManagementController extends AbstractManagementController {
       @PathVariable(name = "id") String id) {
     MemberConfig config = new MemberConfig();
     config.setId(id);
-    ClusterManagementResult result = clusterManagementService.list(config);
-
-    if (result.getResult(RuntimeCacheElement.class).size() == 0) {
-      throw new EntityNotFoundException("Unable to find the member with id = " + id);
-    }
+    ClusterManagementResult result = clusterManagementService.get(config);
 
     return new ResponseEntity<>(result,
         result.isSuccessful() ? HttpStatus.OK : HttpStatus.INTERNAL_SERVER_ERROR);
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
index f8b4aca..5a9f219 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
@@ -18,6 +18,8 @@ package org.apache.geode.management.internal.rest.controllers;
 import static org.apache.geode.cache.configuration.RegionConfig.REGION_CONFIG_ENDPOINT;
 import static org.apache.geode.management.internal.rest.controllers.AbstractManagementController.MANAGEMENT_API_VERSION;
 
+import java.util.List;
+
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
@@ -26,13 +28,20 @@ import org.springframework.http.HttpStatus;
 import org.springframework.http.ResponseEntity;
 import org.springframework.security.access.prepost.PreAuthorize;
 import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
 import org.springframework.web.bind.annotation.RequestBody;
 import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
 
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.management.api.ClusterManagementResult;
+import org.apache.geode.management.configuration.RuntimeIndex;
+import org.apache.geode.management.configuration.RuntimeRegionConfig;
+import org.apache.geode.management.internal.exceptions.EntityNotFoundException;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
 
 @Controller("regionManagement")
 @RequestMapping(MANAGEMENT_API_VERSION)
@@ -56,7 +65,8 @@ public class RegionManagementController extends AbstractManagementController {
 
   @PreAuthorize("@securityService.authorize('CLUSTER', 'READ')")
   @RequestMapping(method = RequestMethod.GET, value = REGION_CONFIG_ENDPOINT)
-  public ResponseEntity<ClusterManagementResult> listRegion(
+  @ResponseBody
+  public ClusterManagementResult listRegion(
       @RequestParam(required = false) String id,
       @RequestParam(required = false) String group) {
     RegionConfig filter = new RegionConfig();
@@ -66,8 +76,54 @@ public class RegionManagementController extends AbstractManagementController {
     if (StringUtils.isNotBlank(group)) {
       filter.setGroup(group);
     }
-    ClusterManagementResult result = clusterManagementService.list(filter);
-    return new ResponseEntity<>(result,
-        result.isSuccessful() ? HttpStatus.OK : HttpStatus.INTERNAL_SERVER_ERROR);
+    return clusterManagementService.list(filter);
+  }
+
+  @RequestMapping(method = RequestMethod.GET, value = REGION_CONFIG_ENDPOINT + "/{id}")
+  @ResponseBody
+  public ClusterManagementResult getRegion(
+      @PathVariable(name = "id") String id) {
+    securityService.authorize(Resource.CLUSTER, Operation.READ, id);
+    RegionConfig config = new RegionConfig();
+    config.setName(id);
+    return clusterManagementService.get(config);
+  }
+
+  @RequestMapping(method = RequestMethod.GET,
+      value = REGION_CONFIG_ENDPOINT + "/{regionName}/indexes")
+  @ResponseBody
+  public ClusterManagementResult listIndex(
+      @PathVariable String regionName,
+      @RequestParam(required = false) String id) {
+
+    ClusterManagementResult result = getRegion(regionName);
+    RuntimeRegionConfig runtimeRegion = result.getResult(RuntimeRegionConfig.class).get(0);
+
+    // only send the index information back
+    List<RuntimeIndex> runtimeIndexes = runtimeRegion.getRuntimeIndexes(id);
+    result.setResult(runtimeIndexes);
+
+    return result;
+  }
+
+  @RequestMapping(method = RequestMethod.GET,
+      value = REGION_CONFIG_ENDPOINT + "/{regionName}/indexes/{id}")
+  @ResponseBody
+  public ClusterManagementResult getIndex(
+      @PathVariable String regionName,
+      @PathVariable String id) {
+    ClusterManagementResult result = listIndex(regionName, id);
+    List<RuntimeIndex> indexList = result.getResult(RuntimeIndex.class);
+
+    if (indexList.size() == 0) {
+      throw new EntityNotFoundException("Index " + id + " not found.");
+    }
+
+    if (indexList.size() > 1) {
+      throw new IllegalStateException("More than one entity found.");
+    }
+
+    result.setResult(indexList);
+    return result;
   }
 }