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

[geode] branch develop updated: GEODE-6779: Create disk store command should only return when the MBeans are available (#3681)

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

jensdeppe 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 f8223f1  GEODE-6779: Create disk store command should only return when the MBeans are available (#3681)
f8223f1 is described below

commit f8223f173a6d7dbc4c9c6f919f629528ab0b1a1e
Author: Jens Deppe <jd...@pivotal.io>
AuthorDate: Mon Jun 10 08:02:43 2019 -0700

    GEODE-6779: Create disk store command should only return when the MBeans are available (#3681)
    
    
    Co-authored-by: Sarah Abbey <sa...@pivotal.io>
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
---
 ...ateRegionWithDiskstoreAndSecurityDUnitTest.java | 12 ++++++--
 .../cli/commands/AlterCompressorDUnitTest.java     |  1 -
 .../CreateAsyncEventQueueCommandDUnitTest.java     |  5 +--
 .../cli/commands/DiskStoreCommandsDUnitTest.java   | 12 +++++---
 .../ClusterConfigImportDUnitTest.java              |  4 +--
 .../cli/commands/CreateDiskStoreCommand.java       | 36 ++++++++++++++++++++++
 .../cli/commands/CreateDiskStoreCommandTest.java   |  2 ++
 .../cli/commands/AlterRegionCommandDUnitTest.java  |  4 ++-
 8 files changed, 62 insertions(+), 14 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
index da2099a..54ee79a 100644
--- 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
@@ -69,7 +69,9 @@ public class CreateRegionWithDiskstoreAndSecurityDUnitTest {
   @Test
   public void createReplicateRegionWithDiskstoreWithoutDataManage() throws Exception {
     gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
-        diskStoreDir.getAbsolutePath())).statusIsSuccess();
+        diskStoreDir.getAbsolutePath()))
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     RegionConfig regionConfig = new RegionConfig();
     regionConfig.setName("REGION1");
@@ -92,7 +94,9 @@ public class CreateRegionWithDiskstoreAndSecurityDUnitTest {
   @Test
   public void createReplicateRegionWithDiskstoreWithoutClusterWrite() throws Exception {
     gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
-        diskStoreDir.getAbsolutePath())).statusIsSuccess();
+        diskStoreDir.getAbsolutePath()))
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     RegionConfig regionConfig = new RegionConfig();
     regionConfig.setName("REGION1");
@@ -115,7 +119,9 @@ public class CreateRegionWithDiskstoreAndSecurityDUnitTest {
   @Test
   public void createReplicateRegionWithDiskstoreSuccess() throws Exception {
     gfsh.executeAndAssertThat(String.format("create disk-store --name=DISKSTORE --dir=%s",
-        diskStoreDir.getAbsolutePath())).statusIsSuccess();
+        diskStoreDir.getAbsolutePath()))
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     RegionConfig regionConfig = new RegionConfig();
     regionConfig.setName("REGION1");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
index c99f54a..aafd0ef 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterCompressorDUnitTest.java
@@ -69,7 +69,6 @@ public class AlterCompressorDUnitTest {
     gfsh.executeAndAssertThat(
         "create disk-store --name=diskStore --groups=dataStore --dir=diskStore").statusIsSuccess();
 
-    locator.waitUntilDiskStoreIsReadyOnExactlyThisManyServers("diskStore", 2);
     // create regions
     gfsh.executeAndAssertThat(
         "create region --name=testRegion --type=REPLICATE_PERSISTENT --group=dataStore --disk-store=diskStore")
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
index 20527e0..6e2d48d 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/CreateAsyncEventQueueCommandDUnitTest.java
@@ -92,8 +92,9 @@ public class CreateAsyncEventQueueCommandDUnitTest {
             " java.lang.IllegalStateException: A GatewaySender with id AsyncEventQueue_queue is already defined in this cache.",
             " java.lang.IllegalStateException: A GatewaySender with id AsyncEventQueue_queue is already defined in this cache.");
 
-    gfsh.executeAndAssertThat("create disk-store --name=diskStore2 --dir=diskstore");
-    locator.waitUntilDiskStoreIsReadyOnExactlyThisManyServers("diskStore2", 2);
+    gfsh.executeAndAssertThat("create disk-store --name=diskStore2 --dir=diskstore")
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     // create another queue with different configuration
     gfsh.executeAndAssertThat(VALID_COMMAND + " --id=queue2 --group=group2 "
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
index ad4721f..2b3cf31 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -87,14 +87,14 @@ public class DiskStoreCommandsDUnitTest implements Serializable {
   private void createDiskStore(MemberVM jmxManager, int serverCount, String group) {
     gfsh.executeAndAssertThat(String.format(
         "create disk-store --name=%s --dir=%s --group=%s --auto-compact=false --compaction-threshold=99 --max-oplog-size=1 --allow-force-compaction=true",
-        DISKSTORE, DISKSTORE, group)).statusIsSuccess();
+        DISKSTORE, DISKSTORE, group))
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     List<String> diskStores =
         IntStream.rangeClosed(1, serverCount).mapToObj(x -> DISKSTORE).collect(Collectors.toList());
     gfsh.executeAndAssertThat("list disk-stores").statusIsSuccess()
         .tableHasColumnWithValuesContaining("Disk Store Name", diskStores.toArray(new String[0]));
-
-    jmxManager.waitUntilDiskStoreIsReadyOnExactlyThisManyServers(DISKSTORE, serverCount);
   }
 
   private static SerializableRunnableIF dataProducer() {
@@ -344,7 +344,8 @@ public class DiskStoreCommandsDUnitTest implements Serializable {
 
     gfsh.executeAndAssertThat(
         String.format("create disk-store --name=%s --dir=%s", DISKSTORE, DISKSTORE))
-        .statusIsSuccess();
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     gfsh.executeAndAssertThat(String.format("destroy disk-store --name=%s --if-exists", DISKSTORE))
         .statusIsSuccess();
@@ -532,7 +533,8 @@ public class DiskStoreCommandsDUnitTest implements Serializable {
     // Create a disk store with the input disk-dir name
     gfsh.executeAndAssertThat(
         String.format("create disk-store --name=%s --dir=%s", DISKSTORE, diskDirectoryName))
-        .statusIsSuccess();
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
 
     // Verify the server defines the disk store with the disk-dir path
     server.invoke(() -> {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
index 7cb6597..3e101d0 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
@@ -96,8 +96,8 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   public void importFailWithExistingDiskStore() {
     lsRule.startServerVM(1, locatorVM.getPort());
     gfshConnector.executeAndAssertThat("create disk-store --name=diskStore1 --dir=testStore")
-        .statusIsSuccess();
-    locatorVM.waitUntilDiskStoreIsReadyOnExactlyThisManyServers("diskStore1", 1);
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
     gfshConnector
         .executeAndAssertThat(
             "import cluster-configuration --zip-file-name=" + clusterConfigZipPath)
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
index 329d149..e741902 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
@@ -20,8 +20,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.logging.log4j.Logger;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
@@ -34,6 +36,8 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.DiskStoreAttributes;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.DistributedSystemMXBean;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.SingleGfshCommand;
@@ -47,6 +51,9 @@ import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission;
 
 public class CreateDiskStoreCommand extends SingleGfshCommand {
+  private static final Logger logger = LogService.getLogger();
+  private static final int MBEAN_CREATION_WAIT_TIME = 10000;
+
   @CliCommand(value = CliStrings.CREATE_DISK_STORE, help = CliStrings.CREATE_DISK_STORE__HELP)
   @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
   @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
@@ -133,10 +140,39 @@ public class CreateDiskStoreCommand extends SingleGfshCommand {
     ResultModel result = ResultModel.createMemberStatusResult(functionResults);
     result.setConfigObject(createDiskStoreType(name, diskStoreAttributes));
 
+    if (!waitForDiskStoreMBeanCreation(name, targetMembers)) {
+      result.addInfo()
+          .addLine("Did not complete waiting for Disk Store MBean proxy creation");
+    }
+
     return result;
   }
 
   @VisibleForTesting
+  boolean waitForDiskStoreMBeanCreation(String diskStore,
+      Set<DistributedMember> membersToCreateDiskStoreOn) {
+    DistributedSystemMXBean dsMXBean = getManagementService().getDistributedSystemMXBean();
+
+    return poll(MBEAN_CREATION_WAIT_TIME, TimeUnit.MILLISECONDS,
+        () -> membersToCreateDiskStoreOn.stream()
+            .allMatch(m -> diskStoreBeanExists(dsMXBean, m.getName(), diskStore)));
+  }
+
+  private boolean diskStoreBeanExists(DistributedSystemMXBean dsMXBean, String memberName,
+      String diskStore) {
+    try {
+      dsMXBean.fetchDiskStoreObjectName(memberName, diskStore);
+      return true;
+    } catch (Exception e) {
+      if (!e.getMessage().toLowerCase().contains("not found")) {
+        logger.warn("Unable to retrieve Disk Store ObjectName for member: {}, diskstore: {} - {}",
+            memberName, diskStore, e.getMessage());
+      }
+    }
+    return false;
+  }
+
+  @VisibleForTesting
   Pair<Boolean, String> validateDiskstoreAttributes(
       DiskStoreAttributes diskStoreAttributes,
       Set<DistributedMember> targetMembers) {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommandTest.java
index a65340c..ac71f7a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommandTest.java
@@ -55,6 +55,7 @@ public class CreateDiskStoreCommandTest {
         .executeAndGetFunctionResult(any(), any(), any());
     doReturn(Pair.of(Boolean.TRUE, null)).when(command).validateDiskstoreAttributes(any(),
         any());
+    doReturn(true).when(command).waitForDiskStoreMBeanCreation(any(), any());
     ResultModel resultModel =
         gfsh.executeAndAssertThat(command, "create disk-store --name=ds1 --dir=./data/persist")
             .getResultModel();
@@ -72,6 +73,7 @@ public class CreateDiskStoreCommandTest {
         .executeAndGetFunctionResult(any(), any(), any());
     doReturn(Pair.of(Boolean.TRUE, null)).when(command).validateDiskstoreAttributes(any(),
         any());
+    doReturn(true).when(command).waitForDiskStoreMBeanCreation(any(), any());
     ResultModel resultModel =
         gfsh.executeAndAssertThat(command, "create disk-store --name=ds1 --dir=/data/persist")
             .getResultModel();
diff --git a/geode-wan/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java b/geode-wan/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
index 1f9a25d..eae2319 100644
--- a/geode-wan/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
+++ b/geode-wan/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRegionCommandDUnitTest.java
@@ -57,7 +57,9 @@ public class AlterRegionCommandDUnitTest {
 
     gfsh.connectAndVerify(locator);
     gfsh.executeAndAssertThat(
-        "create disk-store --name=diskStore --dir=" + temporaryFolder.getRoot()).statusIsSuccess();
+        "create disk-store --name=diskStore --dir=" + temporaryFolder.getRoot())
+        .statusIsSuccess()
+        .doesNotContainOutput("Did not complete waiting");
   }
 
   @Test