You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by js...@apache.org on 2017/09/01 18:36:32 UTC

geode git commit: GEODE-3539: Add tests for List Members and Describe Member

Repository: geode
Updated Branches:
  refs/heads/develop 4ce922095 -> dedfd8ec3


GEODE-3539: Add tests for List Members and Describe Member


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

Branch: refs/heads/develop
Commit: dedfd8ec3fbc140917de876f0345be7e9ddab65f
Parents: 4ce9220
Author: Jared Stewart <js...@pivotal.io>
Authored: Wed Aug 30 11:01:42 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Fri Sep 1 11:35:42 2017 -0700

----------------------------------------------------------------------
 .../cli/commands/ListMemberCommand.java         |  77 -----
 .../cli/commands/ListMembersCommand.java        |  77 +++++
 .../controllers/MemberCommandsController.java   |   3 +-
 .../DescribeMembersCommandDUnitTest.java        |  81 +++++
 .../commands/ListMembersCommandDUnitTest.java   | 123 ++++++++
 .../cli/commands/MemberCommandsDUnitTest.java   | 308 -------------------
 .../internal/security/TestCommand.java          |   2 +-
 7 files changed, 284 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
deleted file mode 100644
index ea88c69..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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 java.util.Set;
-import java.util.TreeSet;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-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.Result;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission;
-
-public class ListMemberCommand implements GfshCommand {
-  @CliCommand(value = {CliStrings.LIST_MEMBER}, help = CliStrings.LIST_MEMBER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.READ)
-  public Result listMember(@CliOption(key = {CliStrings.GROUP}, unspecifiedDefaultValue = "",
-      optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.LIST_MEMBER__GROUP__HELP) String group) {
-    Result result;
-
-    // TODO: Add the code for identifying the system services
-    try {
-      Set<DistributedMember> memberSet = new TreeSet<>();
-      InternalCache cache = getCache();
-
-      // default get all the members in the DS
-      if (group.isEmpty()) {
-        memberSet.addAll(CliUtil.getAllMembers(cache));
-      } else {
-        memberSet.addAll(cache.getDistributedSystem().getGroupMembers(group));
-      }
-
-      if (memberSet.isEmpty()) {
-        result = ResultBuilder.createInfoResult(CliStrings.LIST_MEMBER__MSG__NO_MEMBER_FOUND);
-      } else {
-        TabularResultData resultData = ResultBuilder.createTabularResultData();
-        for (DistributedMember member : memberSet) {
-          resultData.accumulate("Name", member.getName());
-          resultData.accumulate("Id", member.getId());
-        }
-
-        result = ResultBuilder.buildResult(resultData);
-      }
-    } catch (Exception e) {
-      result = ResultBuilder
-          .createGemFireErrorResult("Could not fetch the list of members. " + e.getMessage());
-      LogWrapper.getInstance().warning(e.getMessage(), e);
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMembersCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMembersCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMembersCommand.java
new file mode 100644
index 0000000..34207cd
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMembersCommand.java
@@ -0,0 +1,77 @@
+/*
+ * 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 java.util.Set;
+import java.util.TreeSet;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+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.Result;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.LogWrapper;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class ListMembersCommand implements GfshCommand {
+  @CliCommand(value = {CliStrings.LIST_MEMBER}, help = CliStrings.LIST_MEMBER__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result listMember(@CliOption(key = {CliStrings.GROUP}, unspecifiedDefaultValue = "",
+      optionContext = ConverterHint.MEMBERGROUP,
+      help = CliStrings.LIST_MEMBER__GROUP__HELP) String group) {
+    Result result;
+
+    // TODO: Add the code for identifying the system services
+    try {
+      Set<DistributedMember> memberSet = new TreeSet<>();
+      InternalCache cache = getCache();
+
+      // default get all the members in the DS
+      if (group.isEmpty()) {
+        memberSet.addAll(CliUtil.getAllMembers(cache));
+      } else {
+        memberSet.addAll(cache.getDistributedSystem().getGroupMembers(group));
+      }
+
+      if (memberSet.isEmpty()) {
+        result = ResultBuilder.createInfoResult(CliStrings.LIST_MEMBER__MSG__NO_MEMBER_FOUND);
+      } else {
+        TabularResultData resultData = ResultBuilder.createTabularResultData();
+        for (DistributedMember member : memberSet) {
+          resultData.accumulate("Name", member.getName());
+          resultData.accumulate("Id", member.getId());
+        }
+
+        result = ResultBuilder.buildResult(resultData);
+      }
+    } catch (Exception e) {
+      result = ResultBuilder
+          .createGemFireErrorResult("Could not fetch the list of members. " + e.getMessage());
+      LogWrapper.getInstance().warning(e.getMessage(), e);
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
index ba5c788..c9e7900 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
@@ -21,6 +21,7 @@ import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 
+import org.apache.geode.management.internal.cli.commands.ListMembersCommand;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 
@@ -29,7 +30,7 @@ import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
  * for the Gfsh Member Commands.
  * <p/>
  * 
- * @see org.apache.geode.management.internal.cli.commands.ListMemberCommand
+ * @see org.apache.geode.management.internal.cli.commands.ListMembersCommand
  * @see org.apache.geode.management.internal.cli.commands.DescribeMemberCommand
  * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
  * @see org.springframework.stereotype.Controller

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeMembersCommandDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeMembersCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeMembersCommandDUnitTest.java
new file mode 100644
index 0000000..5a687bb
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DescribeMembersCommandDUnitTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.i18n.CliStrings.DESCRIBE_MEMBER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class DescribeMembersCommandDUnitTest {
+  @ClassRule
+  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+  private static MemberVM locator;
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    locator = lsRule.startLocatorVM(0);
+    lsRule.startServerVM(1, locator.getPort());
+  }
+
+  @Test
+  public void describeInvalidMember() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(DESCRIBE_MEMBER + " --name=foo");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).contains("Member \"foo\" not found");
+  }
+
+  @Test
+  public void describeMembersWhenNotConnected() throws Exception {
+    String result = gfsh.execute(DESCRIBE_MEMBER);
+    assertThat(result)
+        .contains("Command 'describe member' was found but is not currently available");
+  }
+
+  @Test
+  public void describeLocator() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(DESCRIBE_MEMBER + " --name=locator-0");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).contains("locator-0");
+    assertThat(output).doesNotContain("server-1");
+  }
+
+  @Test
+  public void describeServer() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(DESCRIBE_MEMBER + " --name=server-1");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).doesNotContain("locator-0");
+    assertThat(output).contains("server-1");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ListMembersCommandDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ListMembersCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ListMembersCommandDUnitTest.java
new file mode 100644
index 0000000..35b9e4d
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ListMembersCommandDUnitTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.i18n.CliStrings.GROUPS;
+import static org.apache.geode.management.internal.cli.i18n.CliStrings.LIST_MEMBER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Properties;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class ListMembersCommandDUnitTest {
+  @ClassRule
+  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+  private static MemberVM locator;
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    locator = lsRule.startLocatorVM(0, propertiesForGroup("locatorGroup"));
+    lsRule.startServerVM(1, propertiesForGroup("serverGroup1"), locator.getPort());
+    lsRule.startServerVM(2, propertiesForGroup("serverGroup1"), locator.getPort());
+    lsRule.startServerVM(3, propertiesForGroup("serverGroup2"), locator.getPort());
+  }
+
+  @Test
+  public void listMembersWithoutConnection() throws Exception {
+    String result = gfsh.execute(LIST_MEMBER);
+    assertThat(result).contains("Command 'list members' was found but is not currently available");
+  }
+
+  @Test
+  public void listAllMembers() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(LIST_MEMBER);
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).contains("locator-0");
+    assertThat(output).contains("server-1");
+    assertThat(output).contains("server-2");
+    assertThat(output).contains("server-3");
+  }
+
+  @Test
+  public void listMembersInLocatorGroup() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(LIST_MEMBER + " --group=locatorGroup");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).contains("locator-0");
+    assertThat(output).doesNotContain("server-1");
+    assertThat(output).doesNotContain("server-2");
+    assertThat(output).doesNotContain("server-3");
+  }
+
+  @Test
+  public void listMembersInServerGroupOne() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(LIST_MEMBER + " --group=serverGroup1");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).doesNotContain("locator-0");
+    assertThat(output).contains("server-1");
+    assertThat(output).contains("server-2");
+    assertThat(output).doesNotContain("server-3");
+  }
+
+  @Test
+  public void listMembersInServerGroupTwo() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(LIST_MEMBER + " --group=serverGroup2");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).doesNotContain("locator-0");
+    assertThat(output).doesNotContain("server-1");
+    assertThat(output).doesNotContain("server-2");
+    assertThat(output).contains("server-3");
+  }
+
+  @Test
+  public void listMembersInNonExistentGroup() throws Exception {
+    gfsh.connectAndVerify(locator);
+    gfsh.executeAndVerifyCommand(LIST_MEMBER + " --group=foo");
+    String output = gfsh.getGfshOutput();
+
+    assertThat(output).doesNotContain("locator-0");
+    assertThat(output).doesNotContain("server-1");
+    assertThat(output).doesNotContain("server-2");
+    assertThat(output).doesNotContain("server-3");
+    assertThat(output).contains("No Members Found");
+  }
+
+  private static Properties propertiesForGroup(String group) {
+    Properties properties = new Properties();
+    properties.setProperty(GROUPS, group);
+    return properties;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/MemberCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/MemberCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/MemberCommandsDUnitTest.java
deleted file mode 100644
index fe6bc40..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/MemberCommandsDUnitTest.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * 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.distributed.ConfigurationProperties.ENABLE_NETWORK_PARTITION_DETECTION;
-import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_TIME_STATISTICS;
-import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.NAME;
-import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_SAMPLING_ENABLED;
-import static org.apache.geode.test.dunit.Assert.assertEquals;
-import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
-import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.EvictionAttributes;
-import org.apache.geode.cache.FixedPartitionAttributes;
-import org.apache.geode.cache.PartitionAttributes;
-import org.apache.geode.cache.PartitionAttributesFactory;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.Locator;
-import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.cli.Result.Status;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.remote.CommandProcessor;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.SerializableRunnable;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.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;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-@Category(DistributedTest.class)
-public class MemberCommandsDUnitTest extends JUnit4CacheTestCase {
-
-  private static final long serialVersionUID = 1L;
-
-  private static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
-  private static final String REGION1 = "region1";
-  private static final String REGION2 = "region2";
-  private static final String REGION3 = "region3";
-  private static final String SUBREGION1A = "subregion1A";
-  private static final String SUBREGION1B = "subregion1B";
-  private static final String SUBREGION1C = "subregion1C";
-  private static final String PR1 = "PartitionedRegion1";
-  private static final String PR2 = "ParitionedRegion2";
-
-  @ClassRule
-  public static ProvideSystemProperty provideSystemProperty =
-      new ProvideSystemProperty(CliCommandTestBase.USE_HTTP_SYSTEM_PROPERTY, "true");
-
-  @Override
-  public final void postTearDownCacheTestCase() throws Exception {
-    disconnectFromDS();
-  }
-
-  private Properties createProperties(String name, String groups) {
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(LOG_LEVEL, "info");
-    props.setProperty(STATISTIC_SAMPLING_ENABLED, "true");
-    props.setProperty(ENABLE_TIME_STATISTICS, "true");
-    props.setProperty(NAME, name);
-    props.setProperty(GROUPS, groups);
-    return props;
-  }
-
-  private void createRegionsWithSubRegions() {
-    final Cache cache = getCache();
-
-    RegionFactory<String, Integer> dataRegionFactory =
-        cache.createRegionFactory(RegionShortcut.REPLICATE);
-    dataRegionFactory.setConcurrencyLevel(3);
-    Region<String, Integer> region1 = dataRegionFactory.create(REGION1);
-    region1.createSubregion(SUBREGION1C, region1.getAttributes());
-    Region<String, Integer> subregion2 =
-        region1.createSubregion(SUBREGION1A, region1.getAttributes());
-
-    subregion2.createSubregion(SUBREGION1B, subregion2.getAttributes());
-    dataRegionFactory.create(REGION2);
-    dataRegionFactory.create(REGION3);
-  }
-
-  private void createPartitionedRegion1() {
-    final Cache cache = getCache();
-    // Create the data region
-    RegionFactory<String, Integer> dataRegionFactory =
-        cache.createRegionFactory(RegionShortcut.PARTITION);
-    dataRegionFactory.create(PR1);
-  }
-
-  private void createPartitionedRegion(String regionName) {
-    final Cache cache = getCache();
-    // Create the data region
-    RegionFactory<String, Integer> dataRegionFactory =
-        cache.createRegionFactory(RegionShortcut.PARTITION);
-    dataRegionFactory.setConcurrencyLevel(4);
-    EvictionAttributes ea =
-        EvictionAttributes.createLIFOEntryAttributes(100, EvictionAction.LOCAL_DESTROY);
-    dataRegionFactory.setEvictionAttributes(ea);
-    dataRegionFactory.setEnableAsyncConflation(true);
-
-    FixedPartitionAttributes fpa = FixedPartitionAttributes.createFixedPartition("Par1", true);
-    PartitionAttributes pa =
-        new PartitionAttributesFactory().setLocalMaxMemory(100).setRecoveryDelay(2)
-            .setTotalMaxMemory(200).setRedundantCopies(1).addFixedPartitionAttributes(fpa).create();
-    dataRegionFactory.setPartitionAttributes(pa);
-
-    dataRegionFactory.create(regionName);
-  }
-
-
-  private void createLocalRegion() {
-    final Cache cache = getCache();
-    // Create the data region
-    RegionFactory<String, Integer> dataRegionFactory =
-        cache.createRegionFactory(RegionShortcut.LOCAL);
-    dataRegionFactory.create("LocalRegion");
-  }
-
-  private void setupSystem() throws IOException {
-    disconnectAllFromDS();
-    final Host host = Host.getHost(0);
-    final VM[] servers = {host.getVM(0), host.getVM(1)};
-
-    final Properties propsMe = createProperties("me", "G1");
-    final Properties propsServer1 = createProperties("Server1", "G1");
-    final Properties propsServer2 = createProperties("Server2", "G2");
-
-
-    getSystem(propsMe);
-    final Cache cache = getCache();
-    RegionFactory<String, Integer> dataRegionFactory =
-        cache.createRegionFactory(RegionShortcut.REPLICATE_PROXY);
-    dataRegionFactory.setConcurrencyLevel(5);
-    Region<String, Integer> region1 = dataRegionFactory.create(REGION1);
-
-
-    servers[1].invoke(new SerializableRunnable("Create cache for server1") {
-      public void run() {
-        getSystem(propsServer2);
-        createRegionsWithSubRegions();
-        createLocalRegion();
-        createPartitionedRegion("ParReg1");
-      }
-    });
-    servers[0].invoke(new SerializableRunnable("Create cache for server0") {
-      public void run() {
-        getSystem(propsServer1);
-        createRegionsWithSubRegions();
-        createLocalRegion();
-      }
-    });
-  }
-
-  private Properties createProperties(Host host, int locatorPort) {
-    Properties props = new Properties();
-
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(LOCATORS, getServerHostName(host) + "[" + locatorPort + "]");
-    props.setProperty(LOG_LEVEL, "info");
-    props.setProperty(STATISTIC_SAMPLING_ENABLED, "true");
-    props.setProperty(ENABLE_TIME_STATISTICS, "true");
-    props.put(ENABLE_NETWORK_PARTITION_DETECTION, "true");
-
-    return props;
-  }
-
-  /**
-   * Creates the cache.
-   */
-  private void createCache(Properties props) {
-    getSystem(props);
-    final Cache cache = getCache();
-  }
-
-  /**
-   * Tests the execution of "list member" command which should list out all the members in the DS
-   *
-   * @throws IOException
-   * @throws ClassNotFoundException
-   */
-  @Test
-  public void testListMemberAll() throws IOException, ClassNotFoundException {
-    setupSystem();
-    CommandProcessor commandProcessor = new CommandProcessor();
-    Result result =
-        commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
-    String resultOutput = getResultAsString(result);
-    getLogWriter().info(resultOutput);
-    assertEquals(true, result.getStatus().equals(Status.OK));
-    assertTrue(resultOutput.contains("me:"));
-    assertTrue(resultOutput.contains("Server1:"));
-    assertTrue(resultOutput.contains("Server2:"));
-  }
-
-  /**
-   * Tests the execution of "list member" command, when no cache is created
-   *
-   * @throws IOException
-   * @throws ClassNotFoundException
-   */
-  @Test
-  public void testListMemberWithNoCache() throws IOException, ClassNotFoundException {
-    final Host host = Host.getHost(0);
-    final VM[] servers = {host.getVM(0), host.getVM(1)};
-    final int openPorts[] = AvailablePortHelper.getRandomAvailableTCPPorts(1);
-    final File logFile = new File(getUniqueName() + "-locator" + openPorts[0] + ".log");
-
-    Locator locator = Locator.startLocator(openPorts[0], logFile);
-    try {
-
-      final Properties props = createProperties(host, openPorts[0]);
-      CommandProcessor commandProcessor = new CommandProcessor();
-      Result result =
-          commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
-
-      getLogWriter().info("#SB" + getResultAsString(result));
-      assertEquals(true, result.getStatus().equals(Status.ERROR));
-    } finally {
-      locator.stop(); // fix for bug 46562
-    }
-  }
-
-  /**
-   * Tests list member --group=G1
-   *
-   * @throws IOException
-   * @throws ClassNotFoundException
-   */
-  @Test
-  public void testListMemberWithGroups() throws IOException, ClassNotFoundException {
-    setupSystem();
-    CommandProcessor commandProcessor = new CommandProcessor();
-    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_MEMBER);
-    csb.addOption(CliStrings.GROUP, "G1");
-    Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
-    getLogWriter().info("#SB" + getResultAsString(result));
-    assertEquals(true, result.getStatus().equals(Status.OK));
-  }
-
-  /**
-   * Tests the "describe member" command for all the members in the DS
-   *
-   * @throws IOException
-   * @throws ClassNotFoundException
-   */
-  @Test
-  public void testDescribeMember() throws IOException, ClassNotFoundException {
-    setupSystem();
-    CommandProcessor commandProcessor = new CommandProcessor();
-    GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
-    Set<DistributedMember> members = cache.getDistributedSystem().getAllOtherMembers();
-
-    Iterator<DistributedMember> iters = members.iterator();
-
-    while (iters.hasNext()) {
-      DistributedMember member = iters.next();
-      Result result = commandProcessor
-          .createCommandStatement("describe member --name=" + member.getId(), EMPTY_ENV).process();
-      assertEquals(true, result.getStatus().equals(Status.OK));
-      getLogWriter().info("#SB" + getResultAsString(result));
-      // assertIndexDetailsEquals(true, result.getStatus().equals(Status.OK));
-    }
-  }
-
-  private String getResultAsString(Result result) {
-    StringBuilder sb = new StringBuilder();
-    while (result.hasNextLine()) {
-      sb.append(result.nextLine());
-    }
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/dedfd8ec/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index 17719be..aef2238 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -218,7 +218,7 @@ public class TestCommand {
     // createTestCommand("stop locator --name=locator1", clusterManage);
     // createTestCommand("stop server --name=server1", clusterManage);
 
-    // DescribeMemberCommand, ListMemberCommand
+    // DescribeMemberCommand, ListMembersCommand
     createTestCommand("describe member --name=server1", clusterRead);
     createTestCommand("list members", clusterRead);