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 2016/06/27 21:21:14 UTC

[24/50] [abbrv] incubator-geode git commit: GEODE-1574: Remove quotes around negative numbers in command option values

GEODE-1574: Remove quotes around negative numbers in command option values

* Test for CommandRequest
* This closes #172


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/f38d6964
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/f38d6964
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/f38d6964

Branch: refs/heads/feature/GEODE-1571
Commit: f38d6964ba5840ea192db12be2e2ed67d3379dd2
Parents: fc42db8
Author: gmeilen <gr...@gmail.com>
Authored: Tue Jun 21 17:01:28 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 24 11:21:58 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/CommandRequest.java |  22 ++-
 .../management/internal/cli/GfshParser.java     |   8 +-
 .../internal/cli/CommandRequestTest.java        | 161 +++++++++++++++++++
 .../cli/commands/CliCommandTestBase.java        |   2 +-
 .../commands/GemfireDataCommandsDUnitTest.java  |   1 -
 .../DataCommandsOverHttpDistributedTest.java    | 150 +++++++++++++++++
 6 files changed, 339 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
index ffbad51..405925f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
@@ -113,7 +113,19 @@ public class CommandRequest {
   }
 
   public Map<String, String> getParameters() {
-    final Map<String, String> parameters = new HashMap<String, String>(getUserParameters());
+    final Map<String, String> parameters = new HashMap<>();
+    for (Map.Entry<String, String> mapEntry : getUserParameters().entrySet()) {
+      String key = mapEntry.getKey();
+      String value = mapEntry.getValue();
+
+      if (hasQuotesAroundNegativeNumber(value)){
+        String trimmed = value.substring(1, value.length() - 1);
+        parameters.put(key, trimmed);
+      }
+      else {
+        parameters.put(key, value);
+      }
+    }
     parameters.putAll(getCustomParameters());
     return Collections.unmodifiableMap(parameters);
   }
@@ -130,4 +142,12 @@ public class CommandRequest {
     return getParseResult().getParamValueStrings();
   }
 
+  private boolean hasQuotesAroundNegativeNumber(String value) {
+    if (value == null) {
+      return false;
+    } else {
+      return value.startsWith("\"") && value.endsWith("\"") && value.matches("\"-[0-9]+\"");
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
index 9023ebe..4d2564f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
@@ -893,7 +893,6 @@ public class GfshParser implements Parser {
           preConfigureConverters(commandTarget);
 
           try {
-            // TODO: next call invokes HyphenFormatter
             parse = commandTarget.getOptionParser().parse(gfshMethodTarget.getRemainingBuffer());
           } catch (CliException ce) {
             if (ce instanceof CliCommandOptionException) {
@@ -1090,8 +1089,13 @@ public class GfshParser implements Parser {
       }
 
       // Remove outer single or double quotes if found
-      if (string != null && ((string.endsWith("\"") && string.endsWith("\"")) || (string.startsWith("\'") && string.endsWith("\'")))) {
+      boolean hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
+      boolean hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
+
+      while (string != null && (hasDoubleQuotes || hasSingleQuotes)) {
         string = string.substring(1, string.length() - 1);
+        hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
+        hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
       }
 
       if (converter != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
new file mode 100644
index 0000000..8e5d27e
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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 com.gemstone.gemfire.management.internal.cli;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Categories.ExcludeCategory;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class CommandRequestTest {
+
+  private Map<String, String> paramValues;
+  private GfshParseResult mockParseResult;
+  private Map<String, String> mockEnvironment;
+  private CommandRequest commandRequest;
+
+  @Before
+  public void setUp() {
+    this.paramValues = new HashMap<>();
+
+    this.mockParseResult = mock(GfshParseResult.class);
+    when(this.mockParseResult.getUserInput()).thenReturn("rebalance --simulate=true --time-out=-1");
+    when(this.mockParseResult.getParamValueStrings()).thenReturn(this.paramValues);
+
+    this.mockEnvironment = new HashMap<>();
+    this.commandRequest = new CommandRequest(this.mockParseResult, this.mockEnvironment, null);
+  }
+
+  @Test
+  public void getParametersRemovesQuotesAroundNegativeNumbers() {
+    String key = "time-out";
+    String value = "\"-1\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value.substring(1, value.length()-1));
+  }
+
+  @Test
+  public void getParametersWithNullValue() {
+    String key = "key-with-null-value";
+    String value = null;
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isNull();
+  }
+
+  @Test
+  public void getParametersWithEmptyValue() {
+    String key = "key-with-empty-value";
+    String value = "";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEmpty();
+  }
+
+  @Test
+  public void getParametersWithEmptyQuotesValue() {
+    String key = "key-with-empty-quotes-value";
+    String value = "\"\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithNumberValue() {
+    String key = "key-with-number-value";
+    String value = "1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithNegativeNumberValue() {
+    String key = "key-with-negative-number-value";
+    String value = "-1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithHyphenAlphaValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "-A";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithHyphenHyphenNumberValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "--1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithQuotesAndMoreValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "\"-1 this is giberish\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithLotsaQuotesValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "\"\"-1\"\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 1b9d770..ab2ce62 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -356,7 +356,7 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
     CommandResult result = null;
     try {
-      result = (CommandResult) shell.getResult();
+      result = (CommandResult) shell.getResult(); // TODO: this can result in ClassCastException if command resulted in error
     } catch (InterruptedException ex) {
       error("shell received InterruptedException");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index b0272b2..902c3b5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -1856,7 +1856,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Ignore("TODO: enable test after GEODE-1574 is fixed")
   @Test
   public void testSimulateForEntireDSWithTimeout() {
     setupTestRebalanceForEntireDS();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
new file mode 100644
index 0000000..3c673f5
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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 com.gemstone.gemfire.management.internal.cli.commands;
+
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+import static org.junit.Assert.*;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.ProvideSystemProperty;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class DataCommandsOverHttpDistributedTest extends CliCommandTestBase {
+
+  private static final String REBALANCE_REGION_NAME = DataCommandsOverHttpDistributedTest.class.getSimpleName() + "Region";
+
+  @ClassRule
+  public static ProvideSystemProperty provideSystemProperty = new ProvideSystemProperty(CliCommandTestBase.USE_HTTP_SYSTEM_PROPERTY, "true");
+
+  @Test
+  public void testSimulateForEntireDSWithTimeout() {
+    setupTestRebalanceForEntireDS();
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testSimulateForEntireDS verified MBean and executing command");
+
+    String command = "rebalance --simulate=true --time-out=-1";
+
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  SerializableRunnable checkRegionMBeans = new SerializableRunnable() {
+    @Override
+    public void run() {
+      final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          final ManagementService service = ManagementService.getManagementService(getCache());
+          final DistributedRegionMXBean bean = service.getDistributedRegionMXBean(
+            Region.SEPARATOR + REBALANCE_REGION_NAME);
+          if (bean == null) {
+            getLogWriter().info("Still probing for checkRegionMBeans ManagerMBean");
+            return false;
+          } else {
+            // verify that bean is proper before executing tests
+            if (bean.getMembers() != null && bean.getMembers().length > 1 && bean.getMemberCount() > 0 && service.getDistributedSystemMXBean().listRegions().length >= 2) {
+              return true;
+            } else {
+              return false;
+            }
+          }
+        }
+
+        @Override
+        public String description() {
+          return "Probing for testRebalanceCommandForSimulateWithNoMember ManagerMBean";
+        }
+      };
+      waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
+        "/" + REBALANCE_REGION_NAME);
+      assertNotNull(bean);
+    }
+  };
+
+  void setupTestRebalanceForEntireDS() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    setUpJmxManagerOnVm0ThenConnect(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another1");
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 400), "value" + (i + 400));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another2");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+}