You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ad...@apache.org on 2016/12/09 11:23:08 UTC

[1/4] geode git commit: Merge branch 'bugfix/GEODE-1984' into develop

Repository: geode
Updated Branches:
  refs/heads/develop 1d951e334 -> 213d8bc1c


Merge branch 'bugfix/GEODE-1984' into develop


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

Branch: refs/heads/develop
Commit: 213d8bc1c3ad85365d7860e3a5d8ea334691cdd4
Parents: 1d951e3 f522f6c
Author: adongre <ad...@apache.org>
Authored: Fri Dec 9 05:08:36 2016 +0530
Committer: adongre <ad...@apache.org>
Committed: Fri Dec 9 05:08:36 2016 +0530

----------------------------------------------------------------------
 .../internal/AsyncEventQueueImpl.java           |   2 +-
 .../apache/geode/cache/wan/GatewaySender.java   |  13 +
 .../cache/wan/AbstractGatewaySender.java        |   1 +
 .../internal/cli/commands/WanCommands.java      |  59 +-
 .../functions/GatewaySenderDestroyFunction.java |  90 +++
 .../GatewaySenderDestroyFunctionArgs.java       |  33 +
 .../internal/cli/i18n/CliStrings.java           |  18 +
 .../controllers/ShellCommandsController.java    |   2 +
 .../web/controllers/WanCommandsController.java  |  24 +
 .../codeAnalysis/sanctionedSerializables.txt    |   2 +
 .../cli/commands/golden-help-offline.properties |  21 +
 .../cli/commands/golden-help-online.properties  |   6 +
 .../geode/internal/cache/wan/WANTestBase.java   |   2 +-
 .../wan/wancommand/WANCommandTestBase.java      |  33 +
 ...mandCreateDestroyGatewaySenderDUnitTest.java | 794 +++++++++++++++++++
 .../WanCommandCreateGatewaySenderDUnitTest.java | 663 ----------------
 16 files changed, 1092 insertions(+), 671 deletions(-)
----------------------------------------------------------------------



[4/4] geode git commit: GEODE-1984: Fix Issue Make GatewaySender destroy a public API

Posted by ad...@apache.org.
GEODE-1984: Fix Issue Make GatewaySender destroy a public API

    1. Making destory API public
    2. Added a new gfsh WAN command destroyGatewaySender
    3. Impleted GatewaySenderDestroyFunction, which first stop gatewaysender
       and then call destroy on the same.
    4. Added a WAN Rest Controller for destroy gatewaysender
    5. Removed the need of explicit casting of sender to AbstractGatewaySender for calling
       destroy API
    6. Renamed Test WanCommandCreateGatewaySenderDUnitTest to WanCommandCreateDestroyGatewaySenderDUnitTest
    7. Each applicable test now also calls destroy gfsh command and verfies the result
    8. Minor changes in the logging for each tests


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

Branch: refs/heads/develop
Commit: 0eb952a6e5524c66c8520c9e0d8cf09780ef8624
Parents: 1d951e3
Author: adongre <ad...@apache.org>
Authored: Sat Dec 3 19:38:39 2016 +0530
Committer: adongre <ad...@apache.org>
Committed: Fri Dec 9 05:08:36 2016 +0530

----------------------------------------------------------------------
 .../internal/AsyncEventQueueImpl.java           |   2 +-
 .../apache/geode/cache/wan/GatewaySender.java   |  15 +
 .../cache/wan/AbstractGatewaySender.java        |   1 +
 .../internal/cli/commands/WanCommands.java      |  59 +-
 .../functions/GatewaySenderDestroyFunction.java |  90 ++
 .../GatewaySenderDestroyFunctionArgs.java       |  33 +
 .../internal/cli/i18n/CliStrings.java           |  18 +
 .../controllers/ShellCommandsController.java    |   2 +
 .../web/controllers/WanCommandsController.java  |  24 +
 .../codeAnalysis/sanctionedSerializables.txt    |   2 +
 .../cli/commands/golden-help-offline.properties |  21 +
 .../cli/commands/golden-help-online.properties  |   6 +
 .../geode/internal/cache/wan/WANTestBase.java   |   2 +-
 .../wan/wancommand/WANCommandTestBase.java      |  35 +
 ...mandCreateDestroyGatewaySenderDUnitTest.java | 921 +++++++++++++++++++
 .../WanCommandCreateGatewaySenderDUnitTest.java | 663 -------------
 16 files changed, 1223 insertions(+), 671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index a5f0497..2eb53be 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -191,7 +191,7 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
   }
 
   public void destroy() {
-    ((AbstractGatewaySender) this.sender).destroy();
+    this.sender.destroy();
   }
 
   public boolean isBucketSorted() {

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
index d950ef3..f9f8f91 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
@@ -400,4 +400,19 @@ public interface GatewaySender {
 
   public int getMaxParallelismForReplicatedRegion();
 
+
+  /**
+   * Destroys the GatewaySender. Before destroying the sender, caller needs to to ensure that the
+   * sender is stopped so that all the resources (threads, connection pool etc.) will be released
+   * properly. Stopping the sender is not handled in the destroy. Destroy is carried out in
+   * following steps: 1. Take the lifeCycleLock. 2. If the sender is attached to any application
+   * region, throw an exception. 3. Close the GatewaySenderAdvisor. 4. Remove the sender from the
+   * cache. 5. Destroy the region underlying the GatewaySender.
+   * <p>
+   * In case of ParallelGatewaySender, the destroy operation does distributed destroy of the QPR. In
+   * case of SerialGatewaySender, the queue region is destroyed locally.
+   */
+  public void destroy();
+
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
index e1c9010..c4d503e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
@@ -510,6 +510,7 @@ public abstract class AbstractGatewaySender implements GatewaySender, Distributi
    * In case of ParallelGatewaySender, the destroy operation does distributed destroy of the QPR. In
    * case of SerialGatewaySender, the queue region is destroyed locally.
    */
+  @Override
   public void destroy() {
     try {
       this.getLifeCycleLock().writeLock().lock();

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
index db1087d..dd6a327 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
@@ -44,11 +44,7 @@ import org.apache.geode.management.internal.MBeanJMXAdapter;
 import org.apache.geode.management.internal.SystemManagementService;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
-import org.apache.geode.management.internal.cli.functions.GatewayReceiverCreateFunction;
-import org.apache.geode.management.internal.cli.functions.GatewayReceiverFunctionArgs;
-import org.apache.geode.management.internal.cli.functions.GatewaySenderCreateFunction;
-import org.apache.geode.management.internal.cli.functions.GatewaySenderFunctionArgs;
+import org.apache.geode.management.internal.cli.functions.*;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResultException;
 import org.apache.geode.management.internal.cli.result.CompositeResultData;
@@ -1071,6 +1067,57 @@ public class WanCommands implements CommandMarker {
     return result;
   }
 
+
+  @CliCommand(value = CliStrings.DESTROY_GATEWAYSENDER,
+      help = CliStrings.DESTROY_GATEWAYSENDER__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN, writesToSharedConfiguration = true)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
+  public Result destroyGatewaySender(
+      @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__GROUP,
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.DESTROY_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
+              valueSeparator = ",") String[] onGroups,
+      @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__MEMBER,
+          optionContext = ConverterHint.MEMBERIDNAME,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          help = CliStrings.DESTROY_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
+              valueSeparator = ",") String onMember,
+      @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__ID, mandatory = true,
+          optionContext = ConverterHint.GATEWAY_SENDER_ID,
+          help = CliStrings.DESTROY_GATEWAYSENDER__ID__HELP) String id) {
+    Result result = null;
+    try {
+      GatewaySenderDestroyFunctionArgs gatewaySenderDestroyFunctionArgs =
+          new GatewaySenderDestroyFunctionArgs(id);
+
+      Set<DistributedMember> membersToDestroyGatewaySenderOn =
+          CliUtil.findAllMatchingMembers(onGroups, onMember == null ? null : onMember.split(","));
+
+      ResultCollector<?, ?> resultCollector =
+          CliUtil.executeFunction(GatewaySenderDestroyFunction.INSTANCE,
+              gatewaySenderDestroyFunctionArgs, membersToDestroyGatewaySenderOn);
+      @SuppressWarnings("unchecked")
+      List<CliFunctionResult> gatewaySenderDestroyResults =
+          (List<CliFunctionResult>) resultCollector.getResult();
+
+      TabularResultData tabularResultData = ResultBuilder.createTabularResultData();
+      final String errorPrefix = "ERROR: ";
+      for (CliFunctionResult gatewaySenderDestroyResult : gatewaySenderDestroyResults) {
+        boolean success = gatewaySenderDestroyResult.isSuccessful();
+        tabularResultData.accumulate("Member", gatewaySenderDestroyResult.getMemberIdOrName());
+        tabularResultData.accumulate("Status",
+            (success ? "" : errorPrefix) + gatewaySenderDestroyResult.getMessage());
+      }
+      result = ResultBuilder.buildResult(tabularResultData);
+    } catch (IllegalArgumentException e) {
+      LogWrapper.getInstance().info(e.getMessage());
+      result = ResultBuilder.createUserErrorResult(e.getMessage());
+    } catch (CommandResultException crex) {
+      result = handleCommandResultException(crex);
+    }
+    return result;
+  }
+
   private TabularResultData buildReceiverStatus(String memberId, GatewayReceiverMXBean bean,
       TabularResultData resultData) {
     resultData.accumulate(CliStrings.RESULT_HOST_MEMBER, memberId);
@@ -1179,7 +1226,7 @@ public class WanCommands implements CommandMarker {
       CliStrings.STOP_GATEWAYSENDER, CliStrings.CREATE_GATEWAYRECEIVER,
       CliStrings.START_GATEWAYRECEIVER, CliStrings.STOP_GATEWAYRECEIVER, CliStrings.LIST_GATEWAY,
       CliStrings.STATUS_GATEWAYSENDER, CliStrings.STATUS_GATEWAYRECEIVER,
-      CliStrings.LOAD_BALANCE_GATEWAYSENDER})
+      CliStrings.LOAD_BALANCE_GATEWAYSENDER, CliStrings.DESTROY_GATEWAYSENDER})
   public boolean isWanCommandsAvailable() {
     boolean isAvailable = true; // always available on server
     if (CliUtil.isGfshVM()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
new file mode 100644
index 0000000..ba393fe
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
@@ -0,0 +1,90 @@
+/*
+ * 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.functions;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.cache.wan.GatewaySender;
+import org.apache.geode.internal.InternalEntity;
+import org.apache.geode.internal.cache.wan.GatewaySenderException;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.logging.log4j.Logger;
+
+public class GatewaySenderDestroyFunction extends FunctionAdapter implements InternalEntity {
+  private static final long serialVersionUID = 1459761440357690134L;
+
+  private static final Logger logger = LogService.getLogger();
+  private static final String ID = GatewaySenderDestroyFunction.class.getName();
+  public static GatewaySenderDestroyFunction INSTANCE = new GatewaySenderDestroyFunction();
+
+  @Override
+  public void execute(FunctionContext context) {
+    ResultSender<Object> resultSender = context.getResultSender();
+
+    Cache cache = CacheFactory.getAnyInstance();
+    String memberNameOrId =
+        CliUtil.getMemberNameOrId(cache.getDistributedSystem().getDistributedMember());
+
+    GatewaySenderDestroyFunctionArgs gatewaySenderDestroyFunctionArgs =
+        (GatewaySenderDestroyFunctionArgs) context.getArguments();
+
+    try {
+      GatewaySender gatewaySender =
+          cache.getGatewaySender(gatewaySenderDestroyFunctionArgs.getId());
+      if (gatewaySender != null) {
+        gatewaySender.stop();
+        gatewaySender.destroy();
+      } else {
+        throw new GatewaySenderException(
+            "GateWaySender with Id  " + gatewaySenderDestroyFunctionArgs.getId() + " not found");
+      }
+      resultSender.lastResult(new CliFunctionResult(memberNameOrId, true,
+          CliStrings.format(CliStrings.DESTROY_GATEWAYSENDER__MSG__GATEWAYSENDER_0_DESTROYED_ON_1,
+              new Object[] {gatewaySenderDestroyFunctionArgs.getId(), memberNameOrId})));
+
+    } catch (GatewaySenderException gse) {
+      resultSender.lastResult(handleException(memberNameOrId, gse.getMessage(), gse));
+    } catch (Exception e) {
+      String exceptionMsg = e.getMessage();
+      if (exceptionMsg == null) {
+        exceptionMsg = CliUtil.stackTraceAsString(e);
+      }
+      resultSender.lastResult(handleException(memberNameOrId, exceptionMsg, e));
+    }
+  }
+
+  private CliFunctionResult handleException(final String memberNameOrId, final String exceptionMsg,
+      final Exception e) {
+    if (e != null && logger.isDebugEnabled()) {
+      logger.debug(e.getMessage(), e);
+    }
+    if (exceptionMsg != null) {
+      return new CliFunctionResult(memberNameOrId, false, exceptionMsg);
+    }
+
+    return new CliFunctionResult(memberNameOrId);
+  }
+
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs.java
new file mode 100644
index 0000000..cf3fa24
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs.java
@@ -0,0 +1,33 @@
+/*
+ * 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.functions;
+
+import java.io.Serializable;
+
+public class GatewaySenderDestroyFunctionArgs implements Serializable {
+
+  private static final long serialVersionUID = 3848480256348119530L;
+  private String id;
+
+
+  public GatewaySenderDestroyFunctionArgs(String id) {
+    this.id = id;
+  }
+
+  public String getId() {
+    return id;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
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 68dd438..c53b309 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
@@ -2235,6 +2235,24 @@ public class CliStrings {
   public static final String START_GATEWAYSENDER__MEMBER__HELP =
       "Name/Id of the member on which to start the Gateway Sender.";
 
+
+  /* destroy gateway-sender */
+  public static final String DESTROY_GATEWAYSENDER = "destroy gateway-sender";
+  public static final String DESTROY_GATEWAYSENDER__HELP =
+      "Destroy the Gateway Sender on a member or members.";
+  public static final String DESTROY_GATEWAYSENDER__GROUP = "group";
+  public static final String DESTROY_GATEWAYSENDER__GROUP__HELP =
+      "Group(s) of members on which to destroy the Gateway Sender.";
+  public static final String DESTROY_GATEWAYSENDER__MEMBER = "member";
+  public static final String DESTROY_GATEWAYSENDER__MEMBER__HELP =
+      "Name/Id of the member on which to destroy the Gateway Sender.";
+  public static final String DESTROY_GATEWAYSENDER__ID = "id";
+  public static final String DESTROY_GATEWAYSENDER__ID__HELP = "Id of the GatewaySender.";
+  public static final String DESTROY_GATEWAYSENDER__MSG__GATEWAYSENDER_0_DESTROYED_ON_1 =
+      "GatewaySender \"{0}\" destroyed on \"{1}\"";
+
+
+
   /* start gfmon command */
   public static final String START_PULSE = "start pulse";
   public static final String START_PULSE__ERROR =

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
index 577aed6..0ecb77f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
@@ -273,6 +273,8 @@ public class ShellCommandsController extends AbstractCommandsController {
             HttpMethod.POST))
         .add(new Link(CliStrings.CREATE_GATEWAYSENDER, toUri("/gateways/senders", scheme),
             HttpMethod.POST))
+        .add(new Link(CliStrings.DESTROY_GATEWAYSENDER, toUri("/gateways/senders/{id}", scheme),
+            HttpMethod.DELETE))
         .add(new Link(CliStrings.LOAD_BALANCE_GATEWAYSENDER,
             toUri("/gateways/senders/{id}?op=load-balance", scheme), HttpMethod.POST))
         .add(new Link(CliStrings.PAUSE_GATEWAYSENDER,

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
index 03dc5d0..fa5aa57 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
@@ -268,6 +268,30 @@ public class WanCommandsController extends AbstractCommandsController {
     return processCommand(command.toString());
   }
 
+
+  @RequestMapping(method = RequestMethod.DELETE, value = "/gateways/senders/{id}")
+  @ResponseBody
+  public String destroyGatewaySender(@PathVariable("id") final String gatewaySenderId,
+      @RequestParam(value = CliStrings.DESTROY_GATEWAYSENDER__GROUP,
+          required = false) final String[] groups,
+      @RequestParam(value = CliStrings.DESTROY_GATEWAYSENDER__MEMBER,
+          required = false) final String[] members) {
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_GATEWAYSENDER);
+    command.addOption(CliStrings.CREATE_GATEWAYSENDER__ID, gatewaySenderId);
+
+    if (hasValue(groups)) {
+      command.addOption(CliStrings.CREATE_GATEWAYSENDER__GROUP,
+          StringUtils.concat(groups, StringUtils.COMMA_DELIMITER));
+    }
+
+    if (hasValue(members)) {
+      command.addOption(CliStrings.CREATE_GATEWAYSENDER__MEMBER,
+          StringUtils.concat(members, StringUtils.COMMA_DELIMITER));
+    }
+    return processCommand(command.toString());
+  }
+
+
   @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders/{id}",
       params = "op=load-balance")
   @ResponseBody

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
index 3f92148..5ce46bf 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
@@ -569,6 +569,8 @@ org/apache/geode/management/internal/cli/functions/GarbageCollectionFunction,tru
 org/apache/geode/management/internal/cli/functions/GatewayReceiverCreateFunction,true,8746830191680509335
 org/apache/geode/management/internal/cli/functions/GatewayReceiverFunctionArgs,true,-5158224572470173267,bindAddress:java/lang/String,endPort:java/lang/Integer,gatewayTransportFilters:java/lang/String[],manualStart:java/lang/Boolean,maximumTimeBetweenPings:java/lang/Integer,socketBufferSize:java/lang/Integer,startPort:java/lang/Integer
 org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction,true,8746830191680509335
+org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction,true,1459761440357690134
+org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs,true,3848480256348119530,id:java/lang/String
 org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs,true,-5158224572470173267,alertThreshold:java/lang/Integer,batchSize:java/lang/Integer,batchTimeInterval:java/lang/Integer,diskStoreName:java/lang/String,diskSynchronous:java/lang/Boolean,dispatcherThreads:java/lang/Integer,enableBatchConflation:java/lang/Boolean,enablePersistence:java/lang/Boolean,gatewayEventFilters:java/lang/String[],gatewayTransportFilters:java/lang/String[],id:java/lang/String,manualStart:java/lang/Boolean,maxQueueMemory:java/lang/Integer,orderPolicy:java/lang/String,parallel:java/lang/Boolean,remoteDSId:java/lang/Integer,socketBufferSize:java/lang/Integer,socketReadTimeout:java/lang/Integer
 org/apache/geode/management/internal/cli/functions/GetMemberConfigInformationFunction,true,1
 org/apache/geode/management/internal/cli/functions/GetMemberInformationFunction,true,1

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
index 786dd40..f352f3d 100644
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
@@ -835,6 +835,27 @@ PARAMETERS\n\
 \ \ \ \ \ \ \ \ The fully qualified class name of GatewayTransportFilter to be added to the GatewaySender. \n\
 \ \ \ \ \ \ \ \ Required: false\n\
 
+destroy-gateway-sender.help=\
+NAME\n\
+\ \ \ \ destroy gateway-sender\n\
+IS AVAILABLE\n\
+\ \ \ \ false\n\
+SYNOPSIS\n\
+\ \ \ \ Destroy the Gateway Sender on a member or members.\n\
+SYNTAX\n\
+\ \ \ \ destroy gateway-sender --id=value [--group=value(,value)*] [--member=value(,value)*]\n\
+PARAMETERS\n\
+\ \ \ \ group\n\
+\ \ \ \ \ \ \ \ Group(s) of members on which to destroy the Gateway Sender.\n\
+\ \ \ \ \ \ \ \ Required: false\n\
+\ \ \ \ member\n\
+\ \ \ \ \ \ \ \ Name/Id of the member on which to destroy the Gateway Sender.\n\
+\ \ \ \ \ \ \ \ Required: false\n\
+\ \ \ \ id\n\
+\ \ \ \ \ \ \ \ Id of the GatewaySender.\n\
+\ \ \ \ \ \ \ \ Required: true\n\
+
+
 create-index.help=\
 NAME\n\
 \ \ \ \ create index\n\

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
index 57f8f37..2b22650 100644
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
@@ -140,6 +140,12 @@ NAME\n\
 IS AVAILABLE\n\
 \ \ \ \ true\n\
 
+destroy-gateway-sender.help=\
+NAME\n\
+\ \ \ \ destroy gateway-sender\n\
+IS AVAILABLE\n\
+\ \ \ \ true\n\
+
 create-index.help=\
 NAME\n\
 \ \ \ \ create index\n\

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/WANTestBase.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/WANTestBase.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/WANTestBase.java
index 9b9ef2b..b0cece0 100644
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/WANTestBase.java
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/WANTestBase.java
@@ -3418,7 +3418,7 @@ public class WANTestBase extends JUnit4DistributedTestCase {
         break;
       }
     }
-    ((AbstractGatewaySender) sender).destroy();
+    sender.destroy();
   }
 
   public static void verifySenderDestroyed(String senderId, boolean isParallel) {

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
index caa235e..f364ba3 100644
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
@@ -25,7 +25,10 @@ import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
+import org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue;
 import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.internal.cli.commands.CliCommandTestBase;
 import org.apache.geode.test.dunit.*;
@@ -41,6 +44,7 @@ import java.util.Set;
 import static org.apache.geode.distributed.ConfigurationProperties.*;
 import static org.apache.geode.test.dunit.Assert.assertEquals;
 import static org.apache.geode.test.dunit.Assert.fail;
+import static org.junit.Assert.assertNull;
 
 public abstract class WANCommandTestBase extends CliCommandTestBase {
 
@@ -448,6 +452,37 @@ public abstract class WANCommandTestBase extends CliCommandTestBase {
     }
   }
 
+  // Added for gateway destroy command
+  // Copied from WANTestBase.java
+
+  public static void verifySenderDestroyed(String senderId, boolean isParallel) {
+    Set<GatewaySender> senders = cache.getGatewaySenders();
+    AbstractGatewaySender sender = null;
+    for (GatewaySender s : senders) {
+      if (s.getId().equals(senderId)) {
+        sender = (AbstractGatewaySender) s;
+        break;
+      }
+    }
+    assertNull(sender);
+
+    String queueRegionNameSuffix = null;
+    if (isParallel) {
+      queueRegionNameSuffix = ParallelGatewaySenderQueue.QSTRING;
+    } else {
+      queueRegionNameSuffix = "_SERIAL_GATEWAY_SENDER_QUEUE";
+    }
+
+    Set<LocalRegion> allRegions = ((GemFireCacheImpl) cache).getAllRegions();
+    for (LocalRegion region : allRegions) {
+      if (region.getName().indexOf(senderId + queueRegionNameSuffix) != -1) {
+        fail("Region underlying the sender is not destroyed.");
+      }
+    }
+  }
+
+
+
   @Override
   public final void postTearDownCacheTestCase() throws Exception {
     closeCacheAndDisconnect();

http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
new file mode 100644
index 0000000..d095764
--- /dev/null
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
@@ -0,0 +1,921 @@
+/*
+ * 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.internal.cache.wan.wancommand;
+
+import org.apache.geode.cache.wan.GatewaySender;
+import org.apache.geode.cache.wan.GatewaySender.OrderPolicy;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.wan.GatewaySenderException;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.test.dunit.Assert.*;
+import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
+
+@Category(DistributedTest.class)
+public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  private CommandResult executeCommandWithIgnoredExceptions(String command) {
+    final IgnoredException exln = IgnoredException.addIgnoredException("Could not connect");
+    try {
+      CommandResult commandResult = executeCommand(command);
+      return commandResult;
+    } finally {
+      exln.remove();
+    }
+  }
+
+
+  /**
+   * GatewaySender with all default attributes
+   */
+  @Test
+  public void testCreateDestroyGatewaySenderWithDefault() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(
+          "testCreateDestroyGatewaySenderWithDefault stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySenderWithDefault failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", true, false));
+    vm4.invoke(() -> verifySenderState("ln", true, false));
+    vm5.invoke(() -> verifySenderState("ln", true, false));
+
+    // Test Destroy Command.
+    command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(
+          "testCreateDestroyGatewaySenderWithDefault stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+
+    } else {
+      fail("testCreateDestroyGatewaySenderWithDefault failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+  }
+
+  /**
+   * + * GatewaySender with given attribute values +
+   */
+  @Test
+  public void testCreateDestroyGatewaySender() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testCreateDestroyGatewaySender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", false, false));
+    vm4.invoke(() -> verifySenderState("ln", false, false));
+    vm5.invoke(() -> verifySenderState("ln", false, false));
+
+    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
+    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
+    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
+
+    // Test Destroy Command.
+    command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testCreateDestroyGatewaySender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+  }
+
+  /**
+   * GatewaySender with given attribute values. Error scenario where dispatcher threads is set to
+   * more than 1 and no order policy provided.
+   */
+  @Test
+  public void testCreateGatewaySender_Error() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testCreateDestroyGatewaySender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation should fail", status.get(i).indexOf("ERROR:") != -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender failed as did not get CommandResult");
+    }
+
+  }
+
+  /**
+   * GatewaySender with given attribute values and event filters.
+   */
+  @Test
+  public void testCreateDestroyGatewaySenderWithGatewayEventFilters() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER
+        + "=org.apache.geode.cache30.MyGatewayEventFilter1,org.apache.geode.cache30.MyGatewayEventFilter2";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testCreateDestroyGatewaySenderWithGatewayEventFilters stringResult : "
+          + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail(
+          "testCreateDestroyGatewaySenderWithGatewayEventFilters failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", false, false));
+    vm4.invoke(() -> verifySenderState("ln", false, false));
+    vm5.invoke(() -> verifySenderState("ln", false, false));
+
+    List<String> eventFilters = new ArrayList<String>();
+    eventFilters.add("org.apache.geode.cache30.MyGatewayEventFilter1");
+    eventFilters.add("org.apache.geode.cache30.MyGatewayEventFilter2");
+    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
+    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
+    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
+
+    // Test Destroy Command.
+    command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info("testCreateDestroyGatewaySenderWithGatewayEventFilters stringResult : "
+          + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail(
+          "testCreateDestroyGatewaySenderWithGatewayEventFilters failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+
+  }
+
+  /**
+   * GatewaySender with given attribute values and transport filters.
+   */
+  @Test
+  public void testCreateDestroyGatewaySenderWithGatewayTransportFilters() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER
+        + "=org.apache.geode.cache30.MyGatewayTransportFilter1";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySenderWithGatewayTransportFilters stringResult : "
+              + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail(
+          "testCreateDestroyGatewaySenderWithGatewayTransportFilters failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", false, false));
+    vm4.invoke(() -> verifySenderState("ln", false, false));
+    vm5.invoke(() -> verifySenderState("ln", false, false));
+
+    List<String> transportFilters = new ArrayList<String>();
+    transportFilters.add("org.apache.geode.cache30.MyGatewayTransportFilter1");
+    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
+    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
+    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
+
+    // Test Destroy Command.
+    command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySenderWithGatewayTransportFilters stringResult : "
+              + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+
+    } else {
+      fail(
+          "testCreateDestroyGatewaySenderWithGatewayTransportFilters failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+  }
+
+  /**
+   * GatewaySender with given attribute values on given member.
+   */
+  @Test
+  public void testCreateDestroyGatewaySender_OnMember() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    final DistributedMember vm3Member = (DistributedMember) vm3.invoke(() -> getMember());
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__MEMBER + "=" + vm3Member.getId() + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySender_OnMember stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(1, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender_OnMember failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", false, false));
+
+    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
+        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
+
+    // Test Destroy Command.
+    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__MEMBER + "=" + vm3Member.getId();
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySender_OnMember stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(1, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+
+    } else {
+      fail("testCreateDestroyGatewaySender_OnMember failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+  }
+
+  /**
+   * GatewaySender with given attribute values on given group
+   */
+  @Test
+  public void testCreateDestroyGatewaySender_Group() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
+    vm4.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
+    vm5.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__GROUP + "=SenderGroup1" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySender_Group stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(3, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender_Group failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", true, false));
+    vm4.invoke(() -> verifySenderState("ln", true, false));
+    vm5.invoke(() -> verifySenderState("ln", true, false));
+
+    // Test Destroy Command
+    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.DESTROY_GATEWAYSENDER__GROUP + "=SenderGroup1";
+
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyGatewaySender_Group stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(3, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender_Group failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+
+  }
+
+  /**
+   * GatewaySender with given attribute values on given group. Only 2 of 3 members are part of the
+   * group.
+   */
+  @Test
+  public void testCreateDestroyGatewaySender_Group_Scenario2() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
+    vm4.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
+    vm5.invoke(() -> createCacheWithGroups(punePort, "SenderGroup2"));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__GROUP + "=SenderGroup1" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(
+          "testCreateDestroyGatewaySender_Group_Scenario2 stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(2, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender_Group_Scenario2 failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", true, false));
+    vm4.invoke(() -> verifySenderState("ln", true, false));
+
+
+    // Test Destroy Command
+    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.DESTROY_GATEWAYSENDER__GROUP + "=SenderGroup1";
+
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(
+          "testCreateDestroyGatewaySender_Group_Scenario2 stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(2, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyGatewaySender_Group_Scenario2 failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+
+  }
+
+  /**
+   * + * Parallel GatewaySender with given attribute values +
+   */
+  @Test
+  public void testCreateDestroyParallelGatewaySender() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyParallelGatewaySender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender creation failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+    } else {
+      fail("testCreateDestroyParallelGatewaySender failed as did not get CommandResult");
+    }
+
+    vm3.invoke(() -> verifySenderState("ln", false, false));
+    vm4.invoke(() -> verifySenderState("ln", false, false));
+    vm5.invoke(() -> verifySenderState("ln", false, false));
+
+    vm3.invoke(
+        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
+            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
+    vm4.invoke(
+        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
+            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
+    vm5.invoke(
+        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
+            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
+
+    // Test Destroy Command
+    command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter()
+          .info("testCreateDestroyParallelGatewaySender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+
+    } else {
+      fail("testCreateDestroyParallelGatewaySender failed as did not get CommandResult");
+    }
+    vm3.invoke(() -> verifySenderDestroyed("ln", true));
+    vm4.invoke(() -> verifySenderDestroyed("ln", true));
+    vm5.invoke(() -> verifySenderDestroyed("ln", true));
+  }
+
+  /**
+   * Parallel GatewaySender with given attribute values. Provide dispatcherThreads as 2 which is not
+   * valid for Parallel sender.
+   */
+  @Test
+  public void testCreateParallelGatewaySender_Error() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
+    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
+        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
+        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
+    IgnoredException exp =
+        IgnoredException.addIgnoredException(GatewaySenderException.class.getName());
+    try {
+      CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+      if (cmdResult != null) {
+        String strCmdResult = commandResultToString(cmdResult);
+        getLogWriter()
+            .info("testCreateParallelGatewaySender_Error stringResult : " + strCmdResult + ">>>>");
+        assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+        TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+        List<String> status = resultData.retrieveAllValues("Status");
+        assertEquals(5, status.size());
+        for (int i = 0; i < status.size(); i++) {
+          assertTrue("GatewaySender creation should have failed",
+              status.get(i).indexOf("ERROR:") != -1);
+        }
+      } else {
+        fail("testCreateParallelGatewaySender_Error failed as did not get CommandResult");
+      }
+    } finally {
+      exp.remove();
+    }
+
+  }
+
+  @Test
+  public void testDestroyGatewaySender_NotCreatedSender() {
+
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
+
+    Properties props = getDistributedSystemProperties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
+    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
+    setUpJmxManagerOnVm0ThenConnect(props);
+
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
+
+    vm3.invoke(() -> createCache(punePort));
+    vm4.invoke(() -> createCache(punePort));
+    vm5.invoke(() -> createCache(punePort));
+
+    // Test Destroy Command
+    String command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
+    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(
+          "testDestroyGatewaySender_NotCreatedSender stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(5, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy should fail", status.get(i).indexOf("ERROR:") != -1);
+      }
+
+    } else {
+      fail("testCreateDestroyParallelGatewaySender failed as did not get CommandResult");
+    }
+  }
+}


[2/4] geode git commit: GEODE-1984: Addressed review comments, Removed copy pasted repeating code and moved it into a separate method.

Posted by ad...@apache.org.
GEODE-1984: Addressed review comments,
Removed copy pasted repeating code and moved it into a separate method.


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

Branch: refs/heads/develop
Commit: f522f6ca6a3fd362e5d6657ef451743a68a4c0dc
Parents: 0eb952a
Author: adongre <ad...@apache.org>
Authored: Thu Dec 8 12:53:05 2016 +0530
Committer: adongre <ad...@apache.org>
Committed: Fri Dec 9 05:08:36 2016 +0530

----------------------------------------------------------------------
 .../apache/geode/cache/wan/GatewaySender.java   |  16 +-
 .../functions/GatewaySenderDestroyFunction.java |   2 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   2 +-
 .../wan/wancommand/WANCommandTestBase.java      |   4 +-
 ...mandCreateDestroyGatewaySenderDUnitTest.java | 265 +++++--------------
 5 files changed, 79 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/f522f6ca/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
index f9f8f91..1dc1fb1 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
@@ -402,17 +402,15 @@ public interface GatewaySender {
 
 
   /**
-   * Destroys the GatewaySender. Before destroying the sender, caller needs to to ensure that the
-   * sender is stopped so that all the resources (threads, connection pool etc.) will be released
-   * properly. Stopping the sender is not handled in the destroy. Destroy is carried out in
-   * following steps: 1. Take the lifeCycleLock. 2. If the sender is attached to any application
-   * region, throw an exception. 3. Close the GatewaySenderAdvisor. 4. Remove the sender from the
-   * cache. 5. Destroy the region underlying the GatewaySender.
+   * Destroys the GatewaySender.
    * <p>
-   * In case of ParallelGatewaySender, the destroy operation does distributed destroy of the QPR. In
-   * case of SerialGatewaySender, the queue region is destroyed locally.
+   * In case of ParallelGatewaySender, the destroy operation does distributed destroy of the Queue
+   * Region. In case of SerialGatewaySender, the Queue Region is destroyed locally.
+   * 
+   * @since Geode 1.1
+   *
    */
-  public void destroy();
+  void destroy();
 
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/f522f6ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
index ba393fe..2873633 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction.java
@@ -28,7 +28,7 @@ import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.logging.log4j.Logger;
 
 public class GatewaySenderDestroyFunction extends FunctionAdapter implements InternalEntity {
-  private static final long serialVersionUID = 1459761440357690134L;
+  private static final long serialVersionUID = 1L;
 
   private static final Logger logger = LogService.getLogger();
   private static final String ID = GatewaySenderDestroyFunction.class.getName();

http://git-wip-us.apache.org/repos/asf/geode/blob/f522f6ca/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
index 5ce46bf..1aac8c2 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
@@ -569,7 +569,7 @@ org/apache/geode/management/internal/cli/functions/GarbageCollectionFunction,tru
 org/apache/geode/management/internal/cli/functions/GatewayReceiverCreateFunction,true,8746830191680509335
 org/apache/geode/management/internal/cli/functions/GatewayReceiverFunctionArgs,true,-5158224572470173267,bindAddress:java/lang/String,endPort:java/lang/Integer,gatewayTransportFilters:java/lang/String[],manualStart:java/lang/Boolean,maximumTimeBetweenPings:java/lang/Integer,socketBufferSize:java/lang/Integer,startPort:java/lang/Integer
 org/apache/geode/management/internal/cli/functions/GatewaySenderCreateFunction,true,8746830191680509335
-org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction,true,1459761440357690134
+org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunction,true,1
 org/apache/geode/management/internal/cli/functions/GatewaySenderDestroyFunctionArgs,true,3848480256348119530,id:java/lang/String
 org/apache/geode/management/internal/cli/functions/GatewaySenderFunctionArgs,true,-5158224572470173267,alertThreshold:java/lang/Integer,batchSize:java/lang/Integer,batchTimeInterval:java/lang/Integer,diskStoreName:java/lang/String,diskSynchronous:java/lang/Boolean,dispatcherThreads:java/lang/Integer,enableBatchConflation:java/lang/Boolean,enablePersistence:java/lang/Boolean,gatewayEventFilters:java/lang/String[],gatewayTransportFilters:java/lang/String[],id:java/lang/String,manualStart:java/lang/Boolean,maxQueueMemory:java/lang/Integer,orderPolicy:java/lang/String,parallel:java/lang/Boolean,remoteDSId:java/lang/Integer,socketBufferSize:java/lang/Integer,socketReadTimeout:java/lang/Integer
 org/apache/geode/management/internal/cli/functions/GetMemberConfigInformationFunction,true,1

http://git-wip-us.apache.org/repos/asf/geode/blob/f522f6ca/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
index f364ba3..5578f76 100644
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WANCommandTestBase.java
@@ -452,9 +452,6 @@ public abstract class WANCommandTestBase extends CliCommandTestBase {
     }
   }
 
-  // Added for gateway destroy command
-  // Copied from WANTestBase.java
-
   public static void verifySenderDestroyed(String senderId, boolean isParallel) {
     Set<GatewaySender> senders = cache.getGatewaySenders();
     AbstractGatewaySender sender = null;
@@ -473,6 +470,7 @@ public abstract class WANCommandTestBase extends CliCommandTestBase {
       queueRegionNameSuffix = "_SERIAL_GATEWAY_SENDER_QUEUE";
     }
 
+
     Set<LocalRegion> allRegions = ((GemFireCacheImpl) cache).getAllRegions();
     for (LocalRegion region : allRegions) {
       if (region.getName().indexOf(senderId + queueRegionNameSuffix) != -1) {

http://git-wip-us.apache.org/repos/asf/geode/blob/f522f6ca/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
index d095764..8d1f5d8 100644
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateDestroyGatewaySenderDUnitTest.java
@@ -23,11 +23,13 @@ import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 
@@ -95,32 +97,11 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm4.invoke(() -> verifySenderState("ln", true, false));
     vm5.invoke(() -> verifySenderState("ln", true, false));
 
-    // Test Destroy Command.
-    command =
-        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info(
-          "testCreateDestroyGatewaySenderWithDefault stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
+    doDestroyAndVerifyGatewaySender("ln", null, null, "testCreateDestroyGatewaySenderWithDefault",
+        Arrays.asList(vm3, vm4, vm5), 5, false);
+  }
 
-    } else {
-      fail("testCreateDestroyGatewaySenderWithDefault failed as did not get CommandResult");
-    }
 
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
-    vm5.invoke(() -> verifySenderDestroyed("ln", false));
-  }
 
   /**
    * + * GatewaySender with given attribute values +
@@ -186,29 +167,8 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
         1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
 
-    // Test Destroy Command.
-    command =
-        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateDestroyGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateDestroyGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
-    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", null, null, "testCreateDestroyGatewaySender",
+        Arrays.asList(vm3, vm4, vm5), 5, false);
   }
 
   /**
@@ -336,31 +296,9 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
         1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
 
-    // Test Destroy Command.
-    command =
-        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateDestroyGatewaySenderWithGatewayEventFilters stringResult : "
-          + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail(
-          "testCreateDestroyGatewaySenderWithGatewayEventFilters failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
-    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", null, null,
+        "testCreateDestroyGatewaySenderWithGatewayEventFilters", Arrays.asList(vm3, vm4, vm5), 5,
+        false);
 
   }
 
@@ -435,33 +373,9 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
         1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
 
-    // Test Destroy Command.
-    command =
-        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter()
-          .info("testCreateDestroyGatewaySenderWithGatewayTransportFilters stringResult : "
-              + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-
-    } else {
-      fail(
-          "testCreateDestroyGatewaySenderWithGatewayTransportFilters failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
-    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", null, null,
+        "testCreateDestroyGatewaySenderWithGatewayTransportFilters", Arrays.asList(vm3, vm4, vm5),
+        5, false);
   }
 
   /**
@@ -526,29 +440,8 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
         1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
 
-    // Test Destroy Command.
-    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__MEMBER + "=" + vm3Member.getId();
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter()
-          .info("testCreateDestroyGatewaySender_OnMember stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(1, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-
-    } else {
-      fail("testCreateDestroyGatewaySender_OnMember failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", null, vm3Member,
+        "testCreateDestroyGatewaySender_OnMember", Arrays.asList(vm3), 1, false);
   }
 
   /**
@@ -610,31 +503,8 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm4.invoke(() -> verifySenderState("ln", true, false));
     vm5.invoke(() -> verifySenderState("ln", true, false));
 
-    // Test Destroy Command
-    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.DESTROY_GATEWAYSENDER__GROUP + "=SenderGroup1";
-
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter()
-          .info("testCreateDestroyGatewaySender_Group stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(3, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateDestroyGatewaySender_Group failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
-    vm5.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", "SenderGroup1", null,
+        "testCreateDestroyGatewaySender_Group", Arrays.asList(vm3, vm4, vm5), 3, false);
 
   }
 
@@ -697,31 +567,8 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
     vm3.invoke(() -> verifySenderState("ln", true, false));
     vm4.invoke(() -> verifySenderState("ln", true, false));
 
-
-    // Test Destroy Command
-    command = CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.DESTROY_GATEWAYSENDER__GROUP + "=SenderGroup1";
-
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info(
-          "testCreateDestroyGatewaySender_Group_Scenario2 stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(2, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateDestroyGatewaySender_Group_Scenario2 failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderDestroyed("ln", false));
-    vm4.invoke(() -> verifySenderDestroyed("ln", false));
+    doDestroyAndVerifyGatewaySender("ln", "SenderGroup1", null,
+        "testCreateDestroyGatewaySender_Group_Scenario2", Arrays.asList(vm3, vm4), 2, false);
 
   }
 
@@ -791,30 +638,8 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
         () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
             true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
 
-    // Test Destroy Command
-    command =
-        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=ln";
-    cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter()
-          .info("testCreateDestroyParallelGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender destroy failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-
-    } else {
-      fail("testCreateDestroyParallelGatewaySender failed as did not get CommandResult");
-    }
-    vm3.invoke(() -> verifySenderDestroyed("ln", true));
-    vm4.invoke(() -> verifySenderDestroyed("ln", true));
-    vm5.invoke(() -> verifySenderDestroyed("ln", true));
+    doDestroyAndVerifyGatewaySender("ln", null, null, "testCreateDestroyParallelGatewaySender",
+        Arrays.asList(vm3, vm4), 5, true);
   }
 
   /**
@@ -918,4 +743,52 @@ public class WanCommandCreateDestroyGatewaySenderDUnitTest extends WANCommandTes
       fail("testCreateDestroyParallelGatewaySender failed as did not get CommandResult");
     }
   }
+
+  /**
+   * doDestroyAndVerifyGatewaySender helper command.
+   *
+   * @param id if of the Gateway Sender
+   * @param group Group for the GatewaySender
+   * @param member Distributed Member for memeber id.
+   * @param testName testName for the logging
+   * @param vms list of vms where to verify the destroyed gateway sender
+   * @param size command result.
+   * @param isParallel true if parallel , false otherwise.
+   */
+
+  private void doDestroyAndVerifyGatewaySender(final String id, final String group,
+      final DistributedMember member, final String testName, final List<VM> vms, final int size,
+      final boolean isParallel) {
+    String command =
+        CliStrings.DESTROY_GATEWAYSENDER + " --" + CliStrings.DESTROY_GATEWAYSENDER__ID + "=" + id;
+
+    if (group != null) {
+      command += " --" + CliStrings.DESTROY_GATEWAYSENDER__GROUP + "=" + group;
+    }
+
+    if (member != null) {
+      command += " --" + CliStrings.CREATE_GATEWAYSENDER__MEMBER + "=" + member.getId();
+    }
+
+    final CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
+    if (cmdResult != null) {
+      String strCmdResult = commandResultToString(cmdResult);
+      getLogWriter().info(testName + " stringResult : " + strCmdResult + ">>>>");
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
+      List<String> status = resultData.retrieveAllValues("Status");
+      assertEquals(size, status.size());
+      for (int i = 0; i < status.size(); i++) {
+        assertTrue("GatewaySender destroy failed with: " + status.get(i),
+            status.get(i).indexOf("ERROR:") == -1);
+      }
+
+    } else {
+      fail(testName + " failed as did not get CommandResult");
+    }
+    for (VM vm : vms) {
+      vm.invoke(() -> verifySenderDestroyed(id, isParallel));
+    }
+  }
 }


[3/4] geode git commit: GEODE-1984: Fix Issue Make GatewaySender destroy a public API

Posted by ad...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/0eb952a6/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
deleted file mode 100644
index db9aa91..0000000
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
+++ /dev/null
@@ -1,663 +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.internal.cache.wan.wancommand;
-
-import org.apache.geode.cache.wan.GatewaySender;
-import org.apache.geode.cache.wan.GatewaySender.OrderPolicy;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.wan.GatewaySenderException;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.apache.geode.test.dunit.Assert.*;
-import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
-
-@Category(DistributedTest.class)
-public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
-
-  private static final long serialVersionUID = 1L;
-
-  private CommandResult executeCommandWithIgnoredExceptions(String command) {
-    final IgnoredException exln = IgnoredException.addIgnoredException("Could not connect");
-    try {
-      CommandResult commandResult = executeCommand(command);
-      return commandResult;
-    } finally {
-      exln.remove();
-    }
-  }
-
-
-  /**
-   * GatewaySender with all default attributes
-   */
-  @Test
-  public void testCreateGatewaySenderWithDefault() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", true, false));
-    vm4.invoke(() -> verifySenderState("ln", true, false));
-    vm5.invoke(() -> verifySenderState("ln", true, false));
-  }
-
-  /**
-   * GatewaySender with given attribute values
-   */
-  @Test
-  public void testCreateGatewaySender() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", false, false));
-    vm4.invoke(() -> verifySenderState("ln", false, false));
-    vm5.invoke(() -> verifySenderState("ln", false, false));
-
-    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
-    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
-    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
-  }
-
-  /**
-   * GatewaySender with given attribute values. Error scenario where dispatcher threads is set to
-   * more than 1 and no order policy provided.
-   */
-  @Test
-  public void testCreateGatewaySender_Error() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation should fail", status.get(i).indexOf("ERROR:") != -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-  }
-
-  /**
-   * GatewaySender with given attribute values and event filters.
-   */
-  @Test
-  public void testCreateGatewaySenderWithGatewayEventFilters() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER
-        + "=org.apache.geode.cache30.MyGatewayEventFilter1,org.apache.geode.cache30.MyGatewayEventFilter2";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", false, false));
-    vm4.invoke(() -> verifySenderState("ln", false, false));
-    vm5.invoke(() -> verifySenderState("ln", false, false));
-
-    List<String> eventFilters = new ArrayList<String>();
-    eventFilters.add("org.apache.geode.cache30.MyGatewayEventFilter1");
-    eventFilters.add("org.apache.geode.cache30.MyGatewayEventFilter2");
-    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
-    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
-    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, eventFilters, null));
-  }
-
-  /**
-   * GatewaySender with given attribute values and transport filters.
-   */
-  @Test
-  public void testCreateGatewaySenderWithGatewayTransportFilters() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER
-        + "=org.apache.geode.cache30.MyGatewayTransportFilter1";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", false, false));
-    vm4.invoke(() -> verifySenderState("ln", false, false));
-    vm5.invoke(() -> verifySenderState("ln", false, false));
-
-    List<String> transportFilters = new ArrayList<String>();
-    transportFilters.add("org.apache.geode.cache30.MyGatewayTransportFilter1");
-    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
-    vm4.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
-    vm5.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, transportFilters));
-  }
-
-  /**
-   * GatewaySender with given attribute values on given member.
-   */
-  @Test
-  public void testCreateGatewaySender_OnMember() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    final DistributedMember vm3Member = (DistributedMember) vm3.invoke(() -> getMember());
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__MEMBER + "=" + vm3Member.getId() + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(1, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", false, false));
-
-    vm3.invoke(() -> verifySenderAttributes("ln", 2, false, true, 1000, socketReadTimeout, true,
-        1000, 5000, true, false, 1000, 100, 2, OrderPolicy.THREAD, null, null));
-  }
-
-  /**
-   * GatewaySender with given attribute values on given group
-   */
-  @Test
-  public void testCreateGatewaySender_Group() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
-    vm4.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
-    vm5.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__GROUP + "=SenderGroup1" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(3, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", true, false));
-    vm4.invoke(() -> verifySenderState("ln", true, false));
-    vm5.invoke(() -> verifySenderState("ln", true, false));
-  }
-
-  /**
-   * GatewaySender with given attribute values on given group. Only 2 of 3 members are part of the
-   * group.
-   */
-  @Test
-  public void testCreateGatewaySender_Group_Scenario2() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
-    vm4.invoke(() -> createCacheWithGroups(punePort, "SenderGroup1"));
-    vm5.invoke(() -> createCacheWithGroups(punePort, "SenderGroup2"));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__GROUP + "=SenderGroup1" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(2, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", true, false));
-    vm4.invoke(() -> verifySenderState("ln", true, false));
-  }
-
-  /**
-   * Parallel GatewaySender with given attribute values
-   */
-  @Test
-  public void testCreateParallelGatewaySender() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100";
-    CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-    if (cmdResult != null) {
-      String strCmdResult = commandResultToString(cmdResult);
-      getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-      List<String> status = resultData.retrieveAllValues("Status");
-      assertEquals(5, status.size());
-      for (int i = 0; i < status.size(); i++) {
-        assertTrue("GatewaySender creation failed with: " + status.get(i),
-            status.get(i).indexOf("ERROR:") == -1);
-      }
-    } else {
-      fail("testCreateGatewaySender failed as did not get CommandResult");
-    }
-
-    vm3.invoke(() -> verifySenderState("ln", false, false));
-    vm4.invoke(() -> verifySenderState("ln", false, false));
-    vm5.invoke(() -> verifySenderState("ln", false, false));
-
-    vm3.invoke(
-        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
-            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
-    vm4.invoke(
-        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
-            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
-    vm5.invoke(
-        () -> verifySenderAttributes("ln", 2, true, true, 1000, socketReadTimeout, true, 1000, 5000,
-            true, false, 1000, 100, GatewaySender.DEFAULT_DISPATCHER_THREADS, null, null, null));
-  }
-
-  /**
-   * Parallel GatewaySender with given attribute values. Provide dispatcherThreads as 2 which is not
-   * valid for Parallel sender.
-   */
-  @Test
-  public void testCreateParallelGatewaySender_Error() {
-
-    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId(1));
-
-    Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(DISTRIBUTED_SYSTEM_ID, "1");
-    props.setProperty(LOCATORS, "localhost[" + punePort + "]");
-    setUpJmxManagerOnVm0ThenConnect(props);
-
-    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator(2, punePort));
-
-    vm3.invoke(() -> createCache(punePort));
-    vm4.invoke(() -> createCache(punePort));
-    vm5.invoke(() -> createCache(punePort));
-
-    int socketReadTimeout = GatewaySender.MINIMUM_SOCKET_READ_TIMEOUT + 1000;
-    String command = CliStrings.CREATE_GATEWAYSENDER + " --" + CliStrings.CREATE_GATEWAYSENDER__ID
-        + "=ln" + " --" + CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__PARALLEL + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MANUALSTART + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT + "=" + socketReadTimeout + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL + "=5000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE + "=true" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS + "=false" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY + "=1000" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD + "=100" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS + "=2" + " --"
-        + CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY + "=THREAD";
-    IgnoredException exp =
-        IgnoredException.addIgnoredException(GatewaySenderException.class.getName());
-    try {
-      CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
-      if (cmdResult != null) {
-        String strCmdResult = commandResultToString(cmdResult);
-        getLogWriter().info("testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
-        assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-        TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
-        List<String> status = resultData.retrieveAllValues("Status");
-        assertEquals(5, status.size());
-        for (int i = 0; i < status.size(); i++) {
-          assertTrue("GatewaySender creation should have failed",
-              status.get(i).indexOf("ERROR:") != -1);
-        }
-      } else {
-        fail("testCreateGatewaySender failed as did not get CommandResult");
-      }
-    } finally {
-      exp.remove();
-    }
-
-  }
-}