You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@geode.apache.org by GitBox <gi...@apache.org> on 2020/04/21 18:52:08 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #4818: GEODE-7667: Add a 'clear' gfsh command for PR and RR clear

DonalEvans commented on a change in pull request #4818:
URL: https://github.com/apache/geode/pull/4818#discussion_r412366325



##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ClearCommand.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.management.internal.cli.commands.DataCommandsUtils.callFunctionForRegion;
+
+import java.util.Set;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.ConverterHint;
+import org.apache.geode.management.cli.GfshCommand;
+import org.apache.geode.management.internal.cli.domain.DataCommandRequest;
+import org.apache.geode.management.internal.cli.domain.DataCommandResult;
+import org.apache.geode.management.internal.cli.functions.DataCommandFunction;
+import org.apache.geode.management.internal.cli.result.model.ResultModel;
+import org.apache.geode.management.internal.i18n.CliStrings;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+
+public class ClearCommand extends GfshCommand {
+  public static final String REGION_NOT_FOUND = "Region <%s> not found in any of the members";
+
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION})
+  @CliCommand(value = {CliStrings.CLEAR}, help = CliStrings.CLEAR_HELP)
+  public ResultModel clear(
+      @CliOption(key = {CliStrings.CLEAR_REGION_NAME}, mandatory = true,
+          help = CliStrings.CLEAR_REGION_NAME_HELP,
+          optionContext = ConverterHint.REGION_PATH) String regionPath) {
+
+    Cache cache = getCache();
+
+    authorize(Resource.DATA, Operation.WRITE, regionPath);
+
+
+    Region region = cache.getRegion(regionPath);
+    DataCommandFunction clearfn = new DataCommandFunction();
+    DataCommandResult dataResult;
+    if (region == null) {
+      Set<DistributedMember> memberList = findAnyMembersForRegion(regionPath);
+
+      if (CollectionUtils.isEmpty(memberList)) {
+        return new ResultModel().createError(String.format(REGION_NOT_FOUND, regionPath));
+      }
+
+      DataCommandRequest request = new DataCommandRequest();
+      request.setCommand(CliStrings.REMOVE);
+      request.setRemoveAllKeys("ALL");
+      request.setRegionName(regionPath);
+      dataResult = callFunctionForRegion(request, clearfn, memberList);
+    } else {
+      dataResult = clearfn.remove(null, null, regionPath, "ALL",

Review comment:
       Shouldn't the clear command be using the clear function instead of remove?

##########
File path: geode-core/src/main/java/org/apache/geode/management/internal/i18n/CliStrings.java
##########
@@ -1914,9 +1921,10 @@
   public static final String REMOVE__MSG__KEY_EMPTY = "Key is Null";
   public static final String REMOVE__MSG__REGION_NOT_FOUND = "Region <{0}> Not Found";
   public static final String REMOVE__MSG__KEY_NOT_FOUND_REGION = "Key is not present in the region";
-  public static final String REMOVE__MSG__CLEARED_ALL_CLEARS = "Cleared all keys in the region";
-  public static final String REMOVE__MSG__CLEARALL_NOT_SUPPORTED_FOR_PARTITIONREGION =
-      "Option --" + REMOVE__ALL + " is not supported on partitioned region";
+  public static final String REMOVE__MSG__CLEARED_ALL_KEYS = "Cleared all keys in the region";

Review comment:
       It might be best to rename and move this constant to reflect that going forward, only the clear command should be using the message, since remove --all is now deprecated.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
##########
@@ -313,22 +313,21 @@ public DataCommandResult remove(String key, String keyClass, String regionName,
               CliStrings.REMOVE__MSG__KEY_NOT_FOUND_REGION, false);
         }
       } else {
-        DataPolicy policy = region.getAttributes().getDataPolicy();
-        if (!policy.withPartitioning()) {
-          region.clear();
-          if (logger.isDebugEnabled()) {
-            logger.debug("Cleared all keys in the region - {}", regionName);
-          }
-          return DataCommandResult.createRemoveInfoResult(key, null, null,
-              CliStrings.format(CliStrings.REMOVE__MSG__CLEARED_ALL_CLEARS, regionName), true);
-        } else {
-          return DataCommandResult.createRemoveInfoResult(key, null, null,
-              CliStrings.REMOVE__MSG__CLEARALL_NOT_SUPPORTED_FOR_PARTITIONREGION, false);
-        }
+        return clear(region, regionName);
       }
     }
   }
 
+  public DataCommandResult clear(Region region, String regionName) {
+    DataPolicy policy = region.getAttributes().getDataPolicy();

Review comment:
       This variable is never used.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ClearCommand.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.management.internal.cli.commands.DataCommandsUtils.callFunctionForRegion;
+
+import java.util.Set;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.ConverterHint;
+import org.apache.geode.management.cli.GfshCommand;
+import org.apache.geode.management.internal.cli.domain.DataCommandRequest;
+import org.apache.geode.management.internal.cli.domain.DataCommandResult;
+import org.apache.geode.management.internal.cli.functions.DataCommandFunction;
+import org.apache.geode.management.internal.cli.result.model.ResultModel;
+import org.apache.geode.management.internal.i18n.CliStrings;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+
+public class ClearCommand extends GfshCommand {
+  public static final String REGION_NOT_FOUND = "Region <%s> not found in any of the members";
+
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION})
+  @CliCommand(value = {CliStrings.CLEAR}, help = CliStrings.CLEAR_HELP)
+  public ResultModel clear(
+      @CliOption(key = {CliStrings.CLEAR_REGION_NAME}, mandatory = true,
+          help = CliStrings.CLEAR_REGION_NAME_HELP,
+          optionContext = ConverterHint.REGION_PATH) String regionPath) {
+
+    Cache cache = getCache();
+
+    authorize(Resource.DATA, Operation.WRITE, regionPath);
+
+
+    Region region = cache.getRegion(regionPath);
+    DataCommandFunction clearfn = new DataCommandFunction();
+    DataCommandResult dataResult;
+    if (region == null) {
+      Set<DistributedMember> memberList = findAnyMembersForRegion(regionPath);
+
+      if (CollectionUtils.isEmpty(memberList)) {
+        return new ResultModel().createError(String.format(REGION_NOT_FOUND, regionPath));

Review comment:
       `new` is not needed here, since `createError` is a static call.

##########
File path: geode-gfsh/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ClearCommandDUnitTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.management.internal.cli.commands.RemoveCommand.REGION_NOT_FOUND;
+import static org.apache.geode.management.internal.i18n.CliStrings.REMOVE__MSG__CLEARED_ALL_KEYS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.i18n.CliStrings;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.VMProvider;
+
+
+public class ClearCommandDUnitTest {
+  private static final String REPLICATE_REGION_NAME = "replicateRegion";
+  private static final String PARTITIONED_REGION_NAME = "partitionedRegion";
+  private static final String EMPTY_STRING = "";
+  private static final int NUM_ENTRIES = 200;
+
+  @Rule
+  public ClusterStartupRule clusterStartupRule = new ClusterStartupRule();
+
+  @Rule
+  public GfshCommandRule gfsh = new GfshCommandRule();
+
+  private MemberVM locator;
+  private MemberVM server1;
+  private MemberVM server2;
+
+  @Before
+  public void setup() throws Exception {
+    locator = clusterStartupRule.startLocatorVM(0);
+    server1 = clusterStartupRule.startServerVM(1, locator.getPort());
+    server2 = clusterStartupRule.startServerVM(2, locator.getPort());
+
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndAssertThat("create region --name=" + REPLICATE_REGION_NAME + " --type=REPLICATE")
+        .statusIsSuccess();
+    gfsh.executeAndAssertThat(
+        "create region --name=" + PARTITIONED_REGION_NAME + " --type=PARTITION").statusIsSuccess();
+
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/" + REPLICATE_REGION_NAME, 2);
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/" + PARTITIONED_REGION_NAME, 2);
+
+    VMProvider.invokeInEveryMember(ClearCommandDUnitTest::populateTestRegions, server1, server2);
+  }
+
+  private static void populateTestRegions() {
+    Cache cache = CacheFactory.getAnyInstance();
+
+    Region<String, String> replicateRegion = cache.getRegion(REPLICATE_REGION_NAME);
+    replicateRegion.put(EMPTY_STRING, "valueForEmptyKey");
+    for (int i = 0; i < NUM_ENTRIES; i++) {
+      replicateRegion.put("key" + i, "value" + i);
+    }
+
+    Region<String, String> partitionedRegion = cache.getRegion(PARTITIONED_REGION_NAME);
+    replicateRegion.put(EMPTY_STRING, "valueForEmptyKey");
+    for (int i = 0; i < NUM_ENTRIES; i++) {
+      partitionedRegion.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void clearFailsWhenRegionIsNotFound() {
+    String invalidRegionName = "NotAValidRegion";
+    String command = new CommandStringBuilder(CliStrings.CLEAR)
+        .addOption(CliStrings.CLEAR_REGION_NAME, invalidRegionName).getCommandString();
+    gfsh.executeAndAssertThat(command).statusIsError()
+        .containsOutput(String.format(REGION_NOT_FOUND, "/" + invalidRegionName));
+  }
+
+  @Test
+  public void clearSucceedsWithValidReplicateRegion() {
+    String command = new CommandStringBuilder(CliStrings.CLEAR)
+        .addOption(CliStrings.CLEAR_REGION_NAME, REPLICATE_REGION_NAME).getCommandString();
+
+    gfsh.executeAndAssertThat(command).statusIsSuccess();
+
+    assertThat(gfsh.getGfshOutput()).contains(REMOVE__MSG__CLEARED_ALL_KEYS);
+
+    server1.invoke(() -> verifyAllKeysAreRemoved(REPLICATE_REGION_NAME));
+    server2.invoke(() -> verifyAllKeysAreRemoved(REPLICATE_REGION_NAME));
+  }
+
+
+  @Test
+  public void clearSucceedsWithValidPartitionedRegion() {
+    String command = new CommandStringBuilder(CliStrings.CLEAR)
+        .addOption(CliStrings.CLEAR_REGION_NAME, PARTITIONED_REGION_NAME).getCommandString();
+
+    gfsh.executeAndAssertThat(command).statusIsSuccess();
+
+    assertThat(gfsh.getGfshOutput()).contains(REMOVE__MSG__CLEARED_ALL_KEYS);
+
+    server1.invoke(() -> verifyAllKeysAreRemoved(PARTITIONED_REGION_NAME));
+    server2.invoke(() -> verifyAllKeysAreRemoved(PARTITIONED_REGION_NAME));
+  }

Review comment:
       These tests could be replaced with a single parameterized version, which would prevent code duplication, and also allow easy addition of other region types, such as those using persistence, should the test need to be expanded to cover them.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/domain/DataCommandResult.java
##########
@@ -381,6 +397,12 @@ public ResultModel toResultModel() {
     }
 
     ResultModel result = new ResultModel();
+
+    if (warningMessage != null && !warningMessage.isEmpty()) {
+      InfoResultModel info = result.addInfo(HEADER_INFO_SECTION);

Review comment:
       It might be more consistent to either call `setHeader` here instead of `addInfo`, or to rename `HEADER_INFO_SECTION` to `WARNING_INFO_SECTION` or something similar.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
##########
@@ -276,7 +276,7 @@ public DataCommandResult remove(String key, String keyClass, String regionName,
 
     if (StringUtils.isEmpty(removeAllKeys) && (key == null)) {
       return DataCommandResult.createRemoveResult(null, null, null,
-          CliStrings.REMOVE__MSG__KEY_EMPTY, false);
+          "BR" + CliStrings.REMOVE__MSG__KEY_EMPTY, false);

Review comment:
       Remove the "BR" here.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/RemoveCommand.java
##########
@@ -90,7 +90,14 @@ public ResultModel remove(
     }
 
     dataResult.setKeyClass(keyClass);
+    ResultModel result = null;

Review comment:
       Setting the value of `result` here is redundant, since it will always be assigned in the if/else block below.

##########
File path: geode-gfsh/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/ClearCommandDUnitTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.management.internal.cli.commands.RemoveCommand.REGION_NOT_FOUND;
+import static org.apache.geode.management.internal.i18n.CliStrings.REMOVE__MSG__CLEARED_ALL_KEYS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.i18n.CliStrings;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.VMProvider;
+
+
+public class ClearCommandDUnitTest {
+  private static final String REPLICATE_REGION_NAME = "replicateRegion";
+  private static final String PARTITIONED_REGION_NAME = "partitionedRegion";
+  private static final String EMPTY_STRING = "";
+  private static final int NUM_ENTRIES = 200;
+
+  @Rule
+  public ClusterStartupRule clusterStartupRule = new ClusterStartupRule();
+
+  @Rule
+  public GfshCommandRule gfsh = new GfshCommandRule();
+
+  private MemberVM locator;
+  private MemberVM server1;
+  private MemberVM server2;
+
+  @Before
+  public void setup() throws Exception {
+    locator = clusterStartupRule.startLocatorVM(0);
+    server1 = clusterStartupRule.startServerVM(1, locator.getPort());
+    server2 = clusterStartupRule.startServerVM(2, locator.getPort());
+
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndAssertThat("create region --name=" + REPLICATE_REGION_NAME + " --type=REPLICATE")
+        .statusIsSuccess();
+    gfsh.executeAndAssertThat(
+        "create region --name=" + PARTITIONED_REGION_NAME + " --type=PARTITION").statusIsSuccess();
+
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/" + REPLICATE_REGION_NAME, 2);
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/" + PARTITIONED_REGION_NAME, 2);
+
+    VMProvider.invokeInEveryMember(ClearCommandDUnitTest::populateTestRegions, server1, server2);
+  }
+
+  private static void populateTestRegions() {
+    Cache cache = CacheFactory.getAnyInstance();
+
+    Region<String, String> replicateRegion = cache.getRegion(REPLICATE_REGION_NAME);
+    replicateRegion.put(EMPTY_STRING, "valueForEmptyKey");

Review comment:
       Is there a particular reason we put a value with an empty string as the key here? Just curious.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/domain/DataCommandResult.java
##########
@@ -240,6 +242,16 @@ public static DataCommandResult createRemoveResult(Object inputKey, Object value
     return result;
   }
 
+  public static DataCommandResult createClearResult(Throwable error, String errorString,

Review comment:
       This method is currently not being used anywhere.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org