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/20 20:51:46 UTC

[geode] branch develop updated: GEODE-6772: create index will use region to determine the group infor… (#3590)

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 4d2af05  GEODE-6772: create index will use region to determine the group infor… (#3590)
4d2af05 is described below

commit 4d2af053c80a73c39c1965810cfda8bb6bbc68b2
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Mon May 20 13:51:34 2019 -0700

    GEODE-6772: create index will use region to determine the group infor… (#3590)
---
 .../commands/StartServerCommandAcceptanceTest.java |   2 +-
 .../internal/cli/CreateDataSourceCommandTest.java  |   2 +-
 .../internal/cli/DestroyDataSourceCommandTest.java |   6 +-
 .../CreateDefinedIndexesCommandDUnitTest.java      |   4 +-
 .../cli/commands/CreateIndexCommandDUnitTest.java  | 108 ++++++++++++-----
 .../commands/DestroyIndexCommandsDUnitTest.java    |   9 +-
 .../apache/geode/management/cli/GfshCommand.java   |   6 +
 .../internal/cli/commands/CreateIndexCommand.java  | 135 +++++++++++++++++----
 .../management/internal/cli/i18n/CliStrings.java   |   2 +-
 .../internal/cli/remote/CommandExecutor.java       |  25 +---
 .../commands/AlterAsyncEventQueueCommandTest.java  |  14 ++-
 .../cli/commands/CreateIndexCommandTest.java       |  73 ++++++++---
 .../cli/commands/CreateJndiBindingCommandTest.java |   2 +-
 .../commands/DestroyJndiBindingCommandTest.java    |   2 +-
 14 files changed, 278 insertions(+), 112 deletions(-)

diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
index 8c7c01e..b955bf7 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
@@ -123,7 +123,7 @@ public class StartServerCommandAcceptanceTest {
 
     assertThat(startClusterExecution.getOutputText())
         .contains("Successfully connected to: JMX Manager")
-        .contains("Changes to configuration for group 'cluster' are persisted.")
+        .contains("Cluster configuration for group 'cluster' is updated")
         .containsPattern("Server .* " + testName.getMethodName() + " is currently online.");
 
     // Assert Server Properties.
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
index 4395691..0ad8dab 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/CreateDataSourceCommandTest.java
@@ -230,7 +230,7 @@ public class CreateDataSourceCommandTest {
         COMMAND + " --name=name  --url=url")
         .statusIsSuccess()
         .containsOutput("No members found, data source saved to cluster configuration.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), isNotNull());
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyDataSourceCommandTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyDataSourceCommandTest.java
index 159d36c..4388e91 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyDataSourceCommandTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/DestroyDataSourceCommandTest.java
@@ -179,7 +179,7 @@ public class DestroyDataSourceCommandTest {
 
     gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
         .containsOutput("No members found, data source removed from cluster configuration.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(ccService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), isNotNull());
@@ -196,7 +196,7 @@ public class DestroyDataSourceCommandTest {
 
     gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
         .containsOutput("No members found, data source removed from cluster configuration.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(ccService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), isNotNull());
@@ -213,7 +213,7 @@ public class DestroyDataSourceCommandTest {
 
     gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
         .containsOutput("No members found, data source removed from cluster configuration.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(ccService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), isNotNull());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
index 71cb79b..0d75647 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateDefinedIndexesCommandDUnitTest.java
@@ -144,7 +144,7 @@ public class CreateDefinedIndexesCommandDUnitTest {
         .hasTableSection(CreateDefinedIndexesCommand.CREATE_DEFINED_INDEXES_SECTION)
         .hasRowSize(6);
     assertThat(gfsh.getGfshOutput())
-        .contains("Changes to configuration for group 'cluster' are persisted");
+        .contains("Cluster configuration for group 'cluster' is updated");
 
     VMProvider.invokeInEveryMember(() -> {
       Cache cache = ClusterStartupRule.getCache();
@@ -215,7 +215,7 @@ public class CreateDefinedIndexesCommandDUnitTest {
         .statusIsSuccess().containsOutput("Index successfully defined");
 
     gfsh.executeAndAssertThat("create defined indexes --group=group1").statusIsSuccess()
-        .containsOutput("Changes to configuration for group 'group1' are persisted.");
+        .containsOutput("Cluster configuration for group 'group1' is updated");
 
     VMProvider.invokeInEveryMember(() -> {
       Cache cache = ClusterStartupRule.getCache();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandDUnitTest.java
index 31c04c0..77cab59 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandDUnitTest.java
@@ -31,6 +31,8 @@ import org.apache.geode.distributed.internal.InternalConfigurationPersistenceSer
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.assertions.CommandResultAssert;
+import org.apache.geode.test.junit.assertions.TabularResultModelAssert;
 import org.apache.geode.test.junit.categories.OQLIndexTest;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 
@@ -44,15 +46,21 @@ public class CreateIndexCommandDUnitTest {
   @ClassRule
   public static GfshCommandRule gfsh = new GfshCommandRule();
 
-  private static MemberVM locator, server1;
+  private static MemberVM locator, server1, server2;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
     locator = cluster.startLocatorVM(0);
     server1 = cluster.startServerVM(1, locator.getPort());
+    server2 = cluster.startServerVM(2, "group2", locator.getPort());
     gfsh.connectAndVerify(locator);
 
-    // when locator started, the cacheConfig is null
+    // create a region on server-2 in group 2
+    gfsh.executeAndAssertThat("create region --name=regionB --group=group2 --type=REPLICATE")
+        .statusIsSuccess();
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/regionB", 1);
+
+    // cache config in cluster group is null
     locator.invoke(() -> {
       CacheConfig cacheConfig = ClusterStartupRule.getLocator().getConfigurationPersistenceService()
           .getCacheConfig("cluster");
@@ -61,17 +69,22 @@ public class CreateIndexCommandDUnitTest {
   }
 
   @Test
-  public void regionNotExist() {
+  public void regionNotExistInClusterConfig() {
     gfsh.executeAndAssertThat("create index --name=myIndex --expression=id --region=/noExist")
-        .statusIsError().tableHasColumnWithExactValuesInAnyOrder("Status", "ERROR")
-        .tableHasColumnWithExactValuesInAnyOrder("Message", "Region not found : \"/noExist\"");
+        .statusIsError()
+        .hasInfoSection()
+        .hasOutput().contains("Region /noExist does not exist");
+  }
 
-    locator.invoke(() -> {
-      InternalConfigurationPersistenceService configurationService =
-          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
-      CacheConfig cacheConfig = configurationService.getCacheConfig("cluster");
-      assertThat(cacheConfig.findRegionConfiguration("noExist")).isNull();
-    });
+  @Test
+  public void regionNotExistInThatMember() throws Exception {
+    gfsh.executeAndAssertThat(
+        "create index --name=myIndex --expression=id --region=/regionB --member=server-1")
+        .statusIsError()
+        .hasTableSection()
+        .hasRowSize(1)
+        .hasRow(0)
+        .contains("ERROR", "Region not found : \"/regionB\"");
   }
 
   @Test
@@ -83,48 +96,77 @@ public class CreateIndexCommandDUnitTest {
       cache.createRegionFactory(RegionShortcut.REPLICATE).create("regionA");
     });
 
-    // no region exists in cluster config
-    locator.invoke(() -> {
-      InternalConfigurationPersistenceService configurationService =
-          ClusterStartupRule.getLocator().getConfigurationPersistenceService();
-      CacheConfig cacheConfig = configurationService.getCacheConfig("cluster");
-      assertThat(cacheConfig.findRegionConfiguration("regionA")).isNull();
-    });
-
+    // regionA does not exist as far as cluster configuration is concerned
     gfsh.executeAndAssertThat("create index --name=myIndex --expression=id --region=regionA")
-        .statusIsSuccess().tableHasColumnWithValuesContaining("Status", "OK")
-        .tableHasColumnWithExactValuesInAnyOrder("Message", "Index successfully created");
-
-    // after index is created, the cluster config is not udpated with regionA or index
+        .statusIsError()
+        .hasInfoSection()
+        .hasOutput().contains("Region /regionA does not exist");
+
+    // you can only create index on regionA when specifying a --member option
+    gfsh.executeAndAssertThat(
+        "create index --name=myIndex --expression=id --region=regionA --member=server-1")
+        .statusIsSuccess()
+        .hasTableSection()
+        .hasRowSize(1)
+        .hasRow(0)
+        .contains("OK", "Index successfully created");
+
+    // after index is created, the cluster config is not updated with regionA nor index
     locator.invoke(() -> {
       InternalConfigurationPersistenceService configurationService =
           ClusterStartupRule.getLocator().getConfigurationPersistenceService();
-      assertThat(configurationService.getCacheConfig("cluster").findRegionConfiguration("regionA"))
-          .isNull();
+      assertThat(
+          configurationService.getCacheConfig("cluster", true).findRegionConfiguration("regionA"))
+              .isNull();
     });
   }
 
   @Test
-  public void regionExistInClusterConfig() {
-    gfsh.executeAndAssertThat("create region --name=regionB --type=REPLICATE").statusIsSuccess();
-    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/regionB", 1);
+  public void regionExistInGroup2ClusterConfig() {
     locator.invoke(() -> {
       InternalConfigurationPersistenceService configurationService =
           ClusterStartupRule.getLocator().getConfigurationPersistenceService();
-      assertThat(configurationService.getConfiguration("cluster").getCacheXmlContent())
+      assertThat(configurationService.getConfiguration("group2").getCacheXmlContent())
           .contains("<region name=\"regionB\"");
     });
 
-    gfsh.executeAndAssertThat("create index --name=myIndex --expression=id --region=regionB")
-        .statusIsSuccess();
-
+    // make sure index is created on server-2 (which is only in group-2)
+    CommandResultAssert commandAssert =
+        gfsh.executeAndAssertThat("create index --name=myIndex --expression=id --region=regionB")
+            .statusIsSuccess();
+    TabularResultModelAssert createIndexTableAssert =
+        commandAssert.hasTableSection("createIndex");
+    createIndexTableAssert.hasRowSize(1).hasRow(0).contains("OK", "Index successfully created");
+    createIndexTableAssert.hasColumn("Member").asList().first().toString().contains("server-2");
+    commandAssert.containsOutput("Cluster configuration for group 'group2' is updated.");
+
+    // make sure index is inserted in group2's cluster configuration
     locator.invoke(() -> {
       InternalConfigurationPersistenceService configurationService =
           ClusterStartupRule.getLocator().getConfigurationPersistenceService();
-      assertThat(configurationService.getConfiguration("cluster").getCacheXmlContent())
+      assertThat(configurationService.getConfiguration("group2").getCacheXmlContent())
           .contains("<region name=\"regionB\"").contains("<index").contains("expression=\"id\" ")
           .contains("from-clause=\"/regionB\"").contains("name=\"myIndex\"")
           .contains("type=\"range\"");
     });
   }
+
+  @Test
+  public void regionExistInClusterConfigButDifferentGroup() {
+    // regionB is only in group2, not in group1
+    CommandResultAssert commandAssert =
+        gfsh.executeAndAssertThat(
+            "create index --name=index2 --expression=key --region=regionB --group=group1")
+            .statusIsSuccess();
+
+    commandAssert.hasInfoSection().hasOutput().contains("--groups=group1 is ignored.");
+
+    // index is still created and cluster config updated in group2
+    TabularResultModelAssert createIndexTableAssert =
+        commandAssert.hasTableSection("createIndex");
+    createIndexTableAssert.hasRowSize(1).hasRow(0).contains("OK", "Index successfully created");
+    createIndexTableAssert.hasColumn("Member").asList().first().toString().contains("server-2");
+    commandAssert.containsOutput("Cluster configuration for group 'group2' is updated.");
+
+  }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommandsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommandsDUnitTest.java
index 48b96b8..bb94b15 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommandsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexCommandsDUnitTest.java
@@ -35,6 +35,7 @@ import org.apache.geode.management.internal.cli.domain.Stock;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.assertions.CommandResultAssert;
 import org.apache.geode.test.junit.categories.OQLIndexTest;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 
@@ -74,12 +75,16 @@ public class DestroyIndexCommandsDUnitTest {
     gfsh.executeAndAssertThat(String.format("create region --name=%s --type=REPLICATE", REGION_1))
         .statusIsSuccess();
 
-    gfsh.executeAndAssertThat(
+    CommandResultAssert createIndex1Assert = gfsh.executeAndAssertThat(
         String.format("create index --name=%s --expression=key --region=%s", INDEX_1, REGION_1))
         .statusIsSuccess();
-    gfsh.executeAndAssertThat(
+    createIndex1Assert.hasTableSection("createIndex").hasRowSize(2);
+    createIndex1Assert.containsOutput("Cluster configuration for group 'cluster' is updated");
+    CommandResultAssert createIndex2Assert = gfsh.executeAndAssertThat(
         String.format("create index --name=%s --expression=id --region=%s", INDEX_2, REGION_1))
         .statusIsSuccess();
+    createIndex2Assert.hasTableSection("createIndex").hasRowSize(2);
+    createIndex2Assert.containsOutput("Cluster configuration for group 'cluster' is updated");
 
     assertIndexCount(REGION_1, 2);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
index 09b5471..c726dd8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/GfshCommand.java
@@ -33,6 +33,7 @@ import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.ManagementService;
+import org.apache.geode.management.api.ClusterManagementService;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
@@ -87,6 +88,11 @@ public abstract class GfshCommand implements CommandMarker {
     return locator == null ? null : (T) locator.getConfigurationPersistenceService();
   }
 
+  public ClusterManagementService getClusterManagementService() {
+    InternalLocator locator = InternalLocator.getLocator();
+    return locator == null ? null : locator.getClusterManagementService();
+  }
+
   public void setCache(Cache cache) {
     this.cache = (InternalCache) cache;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
index 74718fb..eebc072 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommand.java
@@ -15,29 +15,37 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.management.internal.cli.remote.CommandExecutor.RUN_ON_MEMBER_CHANGE_NOT_PERSISTED;
+import static org.apache.geode.management.internal.cli.remote.CommandExecutor.SERVICE_NOT_RUNNING_CHANGE_NOT_PERSISTED;
+
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import joptsimple.internal.Strings;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
 import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.query.IndexType;
 import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
+import org.apache.geode.management.api.ClusterManagementService;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.cli.SingleGfshCommand;
+import org.apache.geode.management.cli.GfshCommand;
+import org.apache.geode.management.configuration.RuntimeRegionConfig;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.functions.CreateIndexFunction;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.model.InfoResultModel;
 import org.apache.geode.management.internal.cli.result.model.ResultModel;
-import org.apache.geode.management.internal.exceptions.EntityNotFoundException;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission;
 
-public class CreateIndexCommand extends SingleGfshCommand {
+public class CreateIndexCommand extends GfshCommand {
   @Immutable
   private static final CreateIndexFunction createIndexFunction = new CreateIndexFunction();
 
@@ -65,9 +73,34 @@ public class CreateIndexCommand extends SingleGfshCommand {
 
       @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.CREATE_INDEX__GROUP__HELP) final String[] groups) {
-
-    final Set<DistributedMember> targetMembers = findMembers(groups, memberNameOrID);
+          help = CliStrings.CREATE_INDEX__GROUP__HELP) String[] groups) {
+
+    // first find out what groups this region belongs to when using cluster configuration
+    InternalConfigurationPersistenceService ccService = getConfigurationPersistenceService();
+    ClusterManagementService cms = getClusterManagementService();
+    final Set<DistributedMember> targetMembers;
+    ResultModel resultModel = new ResultModel();
+    InfoResultModel info = resultModel.addInfo();
+    String regionName = null;
+    // if cluster management service is enabled and user did not specify a member id, then
+    // we will find the applicable members based on the what group this region is on
+    if (ccService != null && memberNameOrID == null) {
+      regionName = getValidRegionName(regionPath, cms);
+      RuntimeRegionConfig config = getRuntimeRegionConfig(cms, regionName);
+      if (config == null) {
+        return ResultModel.createError("Region " + regionName + " does not exist.");
+      }
+      String[] calculatedGroups = config.getGroups().toArray(new String[0]);
+      if (groups != null && !containsExactlyInAnyOrder(groups, calculatedGroups)) {
+        info.addLine("--groups=" + Strings.join(groups, ",") + " is ignored.");
+      }
+      groups = calculatedGroups;
+      targetMembers = findMembers(groups, null);
+    }
+    // otherwise use the group/members specified in the option to find the applicable members.
+    else {
+      targetMembers = findMembers(groups, memberNameOrID);
+    }
 
     if (targetMembers.isEmpty()) {
       return ResultModel.createError(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
@@ -86,33 +119,83 @@ public class CreateIndexCommand extends SingleGfshCommand {
 
     List<CliFunctionResult> functionResults =
         executeAndGetFunctionResult(createIndexFunction, index, targetMembers);
-    ResultModel result = ResultModel.createMemberStatusResult(functionResults);
-    result.setConfigObject(index);
-    return result;
+    resultModel.addTableAndSetStatus("createIndex", functionResults, true, false);
+
+    if (!resultModel.isSuccessful()) {
+      return resultModel;
+    }
+
+    // update the cluster configuration. Can't use SingleGfshCommand to do the update since in some
+    // cases
+    // groups information is inferred by the region, and the --group option might have the wrong
+    // group
+    // information.
+    if (ccService == null) {
+      info.addLine(SERVICE_NOT_RUNNING_CHANGE_NOT_PERSISTED);
+      return resultModel;
+    }
+    if (memberNameOrID != null) {
+      info.addLine(RUN_ON_MEMBER_CHANGE_NOT_PERSISTED);
+      return resultModel;
+    }
+
+    final InfoResultModel groupStatus = resultModel.addInfo("groupStatus");
+    String finalRegionName = regionName;
+    // at this point, groups should be the regionConfig's groups
+    if (groups.length == 0) {
+      groups = new String[] {"cluster"};
+    }
+    for (String group : groups) {
+      ccService.updateCacheConfig(group, cacheConfig -> {
+        RegionConfig regionConfig = cacheConfig.findRegionConfiguration(finalRegionName);
+        regionConfig.getIndexes().add(index);
+        groupStatus
+            .addLine("Cluster configuration for group '" + group + "' is updated.");
+        return cacheConfig;
+      });
+    }
+    return resultModel;
+  }
+
+  private static boolean containsExactlyInAnyOrder(String[] a, String[] b) {
+    return new HashSet(Arrays.asList(a)).equals(new HashSet(Arrays.asList(b)));
   }
 
-  String getValidRegionName(String regionPath, CacheConfig cacheConfig) {
+  // find a valid regionName when regionPath passed in is in the form of
+  // "/region1.fieldName.fieldName x"
+  // this also handles the possibility when regionName has "." in it, like "/A.B". It's stripping
+  // . part one by one and check if the remaining part is a valid region name or not. If we
+  // could not find a region with any part of the name, (like, couldn't find A.B or A), then A is
+  // returned.
+  String getValidRegionName(String regionPath, ClusterManagementService cms) {
     // Check to see if the region path contains an alias e.g "/region1 r1"
     // Then the first string will be the regionPath
-    String[] regionPathTokens = regionPath.trim().split(" ");
-    regionPath = regionPathTokens[0];
+    String regionName = regionPath.trim().split(" ")[0];
     // check to see if the region path is in the form of "--region=region.entrySet() z"
-    while (regionPath.contains(".") && cacheConfig.findRegionConfiguration(regionPath) == null) {
-      regionPath = regionPath.substring(0, regionPath.lastIndexOf("."));
+    while (regionName.contains(".")) {
+      RuntimeRegionConfig region = getRuntimeRegionConfig(cms, regionName);
+      if (region != null) {
+        break;
+      }
+      // otherwise, strip one more . part off the regionName
+      else {
+        regionName = regionName.substring(0, regionName.lastIndexOf("."));
+      }
     }
-    return regionPath;
+    return regionName;
   }
 
-  @Override
-  public boolean updateConfigForGroup(String group, CacheConfig config, Object element) {
-    RegionConfig.Index index = (RegionConfig.Index) element;
-    String regionPath = getValidRegionName(index.getFromClause(), config);
-
-    RegionConfig regionConfig = config.findRegionConfiguration(regionPath);
-    if (regionConfig == null) {
-      throw new EntityNotFoundException("Region " + index.getFromClause() + " not found.");
+  RuntimeRegionConfig getRuntimeRegionConfig(ClusterManagementService cms,
+      String regionName) {
+    RegionConfig regionConfig = new RegionConfig();
+    regionConfig.setName(regionName);
+    List<RuntimeRegionConfig> list = cms.list(regionConfig).getResult(RuntimeRegionConfig.class);
+    if (list.isEmpty()) {
+      return null;
+    } else {
+      return list.get(0);
     }
-    regionConfig.getIndexes().add(index);
-    return true;
   }
+
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index f6b5d1d..1d4d465 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -738,7 +738,7 @@ public class CliStrings {
   public static final String CREATE_INDEX__TYPE__HELP =
       "Type of the index. Valid values are: range and key. Deprecated: Since Geode 1.4, hash should not be used.";
   public static final String CREATE_INDEX__GROUP__HELP =
-      "Group of members in which the index will be created.";
+      "Group of members in which the index will be created. Value will be ignored when using cluster configuration.";
   public static final String CREATE_INDEX__INVALID__INDEX__TYPE__MESSAGE =
       "Invalid index type. Value must be either range or key.";
   public static final String CREATE_INDEX__SUCCESS__MSG =
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
index 02e15dd..ce400c5 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.remote;
 
-import static org.apache.geode.management.internal.cli.commands.AlterAsyncEventQueueCommand.GROUP_STATUS_SECTION;
-
 import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -37,7 +35,6 @@ import org.apache.geode.management.internal.cli.exceptions.UserErrorException;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.model.InfoResultModel;
 import org.apache.geode.management.internal.cli.result.model.ResultModel;
-import org.apache.geode.management.internal.cli.result.model.TabularResultModel;
 import org.apache.geode.management.internal.exceptions.EntityNotFoundException;
 import org.apache.geode.security.NotAuthorizedException;
 
@@ -176,37 +173,24 @@ public class CommandExecutor {
 
     List<String> groupsToUpdate;
     String groupInput = parseResult.getParamValueAsString("group");
-    TabularResultModel table = null;
 
     if (!StringUtils.isBlank(groupInput)) {
       groupsToUpdate = Arrays.asList(groupInput.split(","));
     } else if (gfshCommand instanceof UpdateAllConfigurationGroupsMarker) {
       groupsToUpdate = ccService.getGroups().stream().collect(Collectors.toList());
-      table = resultModel.addTable(GROUP_STATUS_SECTION);
-      table.setColumnHeader("Group", "Status");
     } else {
       groupsToUpdate = Arrays.asList("cluster");
     }
 
-    final TabularResultModel finalTable = table;
     for (String group : groupsToUpdate) {
       ccService.updateCacheConfig(group, cacheConfig -> {
         try {
           if (gfshCommand.updateConfigForGroup(group, cacheConfig, resultModel.getConfigObject())) {
-            if (finalTable != null) {
-              finalTable.addRow(group, "Cluster Configuration Updated");
-            } else {
-              infoResultModel
-                  .addLine("Changes to configuration for group '" + group + "' are persisted.");
-            }
+            infoResultModel
+                .addLine("Cluster configuration for group '" + group + "' is updated.");
           } else {
-            if (finalTable != null) {
-              finalTable.addRow(group, "Cluster Configuration not updated");
-
-            } else {
-              infoResultModel
-                  .addLine("No changes were made to the configuration for group '" + group + "'");
-            }
+            infoResultModel
+                .addLine("Cluster configuration for group '" + group + "' is not updated.");
           }
         } catch (Exception e) {
           String message = "Failed to update cluster config for " + group;
@@ -216,7 +200,6 @@ public class CommandExecutor {
           infoResultModel.addLine(message + ". Reason: " + e.getMessage());
           return null;
         }
-
         return cacheConfig;
       });
     }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
index 1e9144a..b6d5735 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/AlterAsyncEventQueueCommandTest.java
@@ -119,8 +119,9 @@ public class AlterAsyncEventQueueCommandTest {
   @Test
   public void queueIdFoundInTheMap_updateBatchSize() throws Exception {
     gfsh.executeAndAssertThat(command, "alter async-event-queue --batch-size=100 --id=queue1")
-        .statusIsSuccess().tableHasRowCount(2)
-        .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
+        .statusIsSuccess()
+        .containsOutput("Cluster configuration for group 'group1' is updated")
+        .containsOutput("Cluster configuration for group 'group2' is not updated")
         .containsOutput("Please restart the servers to apply any changed configuration");
   }
 
@@ -129,8 +130,8 @@ public class AlterAsyncEventQueueCommandTest {
     gfsh.executeAndAssertThat(command,
         "alter async-event-queue --batch-time-interval=100 --id=queue1")
         .statusIsSuccess()
-        .tableHasRowCount(2)
-        .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
+        .containsOutput("Cluster configuration for group 'group1' is updated")
+        .containsOutput("Cluster configuration for group 'group2' is not updated")
         .containsOutput("Please restart the servers to apply any changed configuration");
 
     gfsh.executeAndAssertThat(command,
@@ -141,8 +142,9 @@ public class AlterAsyncEventQueueCommandTest {
   @Test
   public void queueIdFoundInTheMap_updateMaxMemory() throws Exception {
     gfsh.executeAndAssertThat(command, "alter async-event-queue --max-queue-memory=100 --id=queue1")
-        .statusIsSuccess().tableHasRowCount(2)
-        .tableHasRowWithValues("Group", "Status", "group1", "Cluster Configuration Updated")
+        .statusIsSuccess()
+        .containsOutput("Cluster configuration for group 'group1' is updated")
+        .containsOutput("Cluster configuration for group 'group2' is not updated")
         .containsOutput("Please restart the servers to apply any changed configuration");
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandTest.java
index cb7ff7b..2fb1c9e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateIndexCommandTest.java
@@ -24,17 +24,23 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.List;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-import org.apache.geode.cache.configuration.CacheConfig;
 import org.apache.geode.cache.configuration.RegionConfig;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
+import org.apache.geode.management.api.ClusterManagementResult;
+import org.apache.geode.management.api.ClusterManagementService;
+import org.apache.geode.management.configuration.RuntimeRegionConfig;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.test.junit.rules.GfshParserRule;
 
@@ -45,6 +51,8 @@ public class CreateIndexCommandTest {
   private CreateIndexCommand command;
   private CommandResult result;
   private ResultCollector rc;
+  private InternalConfigurationPersistenceService ccService;
+  private ClusterManagementService cms;
 
   @Before
   public void before() throws Exception {
@@ -53,6 +61,8 @@ public class CreateIndexCommandTest {
     when(rc.getResult()).thenReturn(Collections.emptyList());
     doReturn(Collections.emptyList()).when(command).executeAndGetFunctionResult(any(), any(),
         any());
+    ccService = mock(InternalConfigurationPersistenceService.class);
+    cms = mock(ClusterManagementService.class);
   }
 
   @Test
@@ -95,6 +105,14 @@ public class CreateIndexCommandTest {
   }
 
   @Test
+  public void validRegionPath() throws Exception {
+    doReturn(ccService).when(command).getConfigurationPersistenceService();
+    gfshParser.executeAndAssertThat(command,
+        "create index --name=abc --expression=abc --region=\"region.entrySet() z\" --type=range")
+        .statusIsError();
+  }
+
+  @Test
   public void validIndexType2() throws Exception {
     doReturn(Collections.EMPTY_SET).when(command).findMembers(any(), any());
     gfshParser.executeAndAssertThat(command,
@@ -113,7 +131,7 @@ public class CreateIndexCommandTest {
   }
 
   @Test
-  public void defaultInexType() throws Exception {
+  public void defaultIndexType() throws Exception {
     DistributedMember member = mock(DistributedMember.class);
     doReturn(Collections.singleton(member)).when(command).findMembers(any(), any());
 
@@ -130,18 +148,45 @@ public class CreateIndexCommandTest {
 
   @Test
   public void getValidRegionName() {
-    CacheConfig cacheConfig = mock(CacheConfig.class);
-    RegionConfig region = new RegionConfig("regionA.regionB", "REPLICATE");
-    when(cacheConfig.findRegionConfiguration("/regionA.regionB")).thenReturn(region);
-
-    assertThat(command.getValidRegionName("regionB", cacheConfig)).isEqualTo("regionB");
-    assertThat(command.getValidRegionName("/regionB", cacheConfig)).isEqualTo("/regionB");
-    assertThat(command.getValidRegionName("/regionB b", cacheConfig)).isEqualTo("/regionB");
-    assertThat(command.getValidRegionName("/regionB.entrySet()", cacheConfig))
-        .isEqualTo("/regionB");
-    assertThat(command.getValidRegionName("/regionA.regionB.entrySet() A", cacheConfig))
-        .isEqualTo("/regionA.regionB");
-    assertThat(command.getValidRegionName("/regionB.regionA.entrySet() B", cacheConfig))
+    // the existing configuration has a region named /regionA.B
+    doReturn(mock(RuntimeRegionConfig.class)).when(command).getRuntimeRegionConfig(cms,
+        "/regionA.B");
+    when(cms.list(any())).thenReturn(new ClusterManagementResult());
+
+    assertThat(command.getValidRegionName("regionB", cms)).isEqualTo("regionB");
+    assertThat(command.getValidRegionName("/regionB", cms)).isEqualTo("/regionB");
+    assertThat(command.getValidRegionName("/regionB b", cms)).isEqualTo("/regionB");
+    assertThat(command.getValidRegionName("/regionB.entrySet()", cms))
         .isEqualTo("/regionB");
+    assertThat(command.getValidRegionName("/regionA.B.entrySet() A", cms))
+        .isEqualTo("/regionA.B");
+    assertThat(command.getValidRegionName("/regionA.fieldName.entrySet() B", cms))
+        .isEqualTo("/regionA");
+  }
+
+  @Test
+  public void groupIgnored() throws Exception {
+    doReturn(ccService).when(command).getConfigurationPersistenceService();
+    RuntimeRegionConfig config = mock(RuntimeRegionConfig.class);
+    List<String> realGroups = Arrays.asList("group2", "group1");
+    when(config.getGroups()).thenReturn(realGroups);
+    doReturn(config).when(command).getRuntimeRegionConfig(any(), any());
+
+    doReturn(Collections.singleton(mock(DistributedMember.class))).when(command).findMembers(any(),
+        any());
+
+    CliFunctionResult result = new CliFunctionResult("member", false, "reason");
+    doReturn(Collections.singletonList(result)).when(command).executeAndGetFunctionResult(any(),
+        any(), any());
+
+    gfshParser.executeAndAssertThat(command,
+        "create index --name=index --expression=abc --region=/regionA --groups=group1,group2")
+        .statusIsError()
+        .doesNotContainOutput("--groups=group1,group2 is ignored");
+
+    gfshParser.executeAndAssertThat(command,
+        "create index --name=index --expression=abc --region=/regionA --groups=group1,group3")
+        .statusIsError()
+        .containsOutput("--groups=group1,group3 is ignored");
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
index 378466d..cac25a8 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
@@ -215,7 +215,7 @@ public class CreateJndiBindingCommandTest {
     gfsh.executeAndAssertThat(command,
         COMMAND + " --type=SIMPLE --name=name --jdbc-driver-class=driver --connection-url=url")
         .statusIsSuccess().containsOutput("No members found.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(clusterConfigService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
index e61805a..8461d3c 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
@@ -130,7 +130,7 @@ public class DestroyJndiBindingCommandTest {
 
     gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
         .containsOutput("No members found.")
-        .containsOutput("Changes to configuration for group 'cluster' are persisted.");
+        .containsOutput("Cluster configuration for group 'cluster' is updated");
 
     verify(ccService).updateCacheConfig(any(), any());
     verify(command).updateConfigForGroup(eq("cluster"), eq(cacheConfig), any());