You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by sa...@apache.org on 2018/03/08 19:43:52 UTC

[geode] branch develop updated: GEODE-4384: gfsh command to destroy jndi binding (#1568)

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

sai_boorlagadda pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 06e7029  GEODE-4384: gfsh command to destroy jndi binding (#1568)
06e7029 is described below

commit 06e7029ca487636d4e3582fa9eb719a77f65a9f1
Author: Sai Boorlagadda <sa...@gmail.com>
AuthorDate: Thu Mar 8 11:43:49 2018 -0800

    GEODE-4384: gfsh command to destroy jndi binding (#1568)
---
 .../internal/ClusterConfigurationService.java      |  25 +++
 .../apache/geode/internal/jndi/JNDIInvoker.java    |  14 ++
 .../cli/commands/CommandAvailabilityIndicator.java |   2 +-
 .../cli/commands/CreateJndiBindingCommand.java     |  30 +--
 .../cli/commands/DestroyJndiBindingCommand.java    | 119 ++++++++++
 .../cli/functions/DestroyJndiBindingFunction.java  |  44 ++++
 .../sanctioned-geode-core-serializables.txt        |   1 +
 .../cli/commands/CreateJndiBindingCommandTest.java |  80 +------
 .../DestroyJndiBindingCommandDUnitTest.java        | 107 +++++++++
 .../commands/DestroyJndiBindingCommandTest.java    | 244 +++++++++++++++++++++
 .../functions/DestroyJndiBindingFunctionTest.java  |  96 ++++++++
 11 files changed, 669 insertions(+), 93 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
index c81ca54..d111dcc 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
@@ -59,6 +59,8 @@ import org.apache.commons.io.filefilter.DirectoryFileFilter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 import org.apache.geode.CancelException;
@@ -176,6 +178,29 @@ public class ClusterConfigurationService {
   }
 
   /**
+   * Finds xml element in a group's xml, with the tagName that has given attribute and value
+   */
+  public Element getXmlElement(String group, String tagName, String attribute, String value)
+      throws IOException, SAXException, ParserConfigurationException {
+    if (group == null) {
+      group = "cluster";
+    }
+    Configuration config = getConfiguration(group);
+    Document document = XmlUtils.createDocumentFromXml(config.getCacheXmlContent());
+    NodeList elements = document.getElementsByTagName(tagName);
+    if (elements == null || elements.getLength() == 0) {
+      return null;
+    } else {
+      for (int i = 0; i < elements.getLength(); i++) {
+        Element eachElement = (Element) elements.item(i);
+        if (eachElement.getAttribute(attribute).equals(value))
+          return eachElement;
+      }
+    }
+    return null;
+  }
+
+  /**
    * Adds/replaces the xml entity in the shared configuration we don't need to trigger the change
    * listener for this modification, so it's ok to operate on the original configuration object
    */
diff --git a/geode-core/src/main/java/org/apache/geode/internal/jndi/JNDIInvoker.java b/geode-core/src/main/java/org/apache/geode/internal/jndi/JNDIInvoker.java
index f439047..6efb2a8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/jndi/JNDIInvoker.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/jndi/JNDIInvoker.java
@@ -18,6 +18,7 @@ import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Hashtable;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -360,6 +361,19 @@ public class JNDIInvoker {
     }
   }
 
+  public static void unMapDatasource(String jndiName) throws NamingException {
+    ctx.unbind("java:/" + jndiName);
+    for (Iterator it = dataSourceList.iterator(); it.hasNext();) {
+      Object obj = it.next();
+      if (obj instanceof AbstractDataSource) {
+        ((AbstractDataSource) obj).clearUp();
+      } else if (obj instanceof ClientConnectionFactoryWrapper) {
+        ((ClientConnectionFactoryWrapper) obj).clearUp();
+      }
+      it.remove();
+    }
+  }
+
   /**
    * @return Context the existing JNDI Context. If there is no pre-esisting JNDI Context, the
    *         GemFire JNDI Context is returned.
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CommandAvailabilityIndicator.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CommandAvailabilityIndicator.java
index f6939f3..dfd56f8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CommandAvailabilityIndicator.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CommandAvailabilityIndicator.java
@@ -50,7 +50,7 @@ public class CommandAvailabilityIndicator implements CommandMarker {
       CliStrings.STOP_GATEWAYRECEIVER, CliStrings.LIST_GATEWAY, CliStrings.STATUS_GATEWAYSENDER,
       CliStrings.STATUS_GATEWAYRECEIVER, CliStrings.LOAD_BALANCE_GATEWAYSENDER,
       CliStrings.DESTROY_GATEWAYSENDER, AlterAsyncEventQueueCommand.COMMAND_NAME,
-      CreateJndiBindingCommand.CREATE_JNDIBINDING})
+      CreateJndiBindingCommand.CREATE_JNDIBINDING, DestroyJndiBindingCommand.CREATE_JNDIBINDING})
   public boolean clientCommandsAvailable() {
     Gfsh gfsh = Gfsh.getCurrentInstance();
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
index c8459f7..c93d425 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommand.java
@@ -162,10 +162,13 @@ public class CreateJndiBindingCommand implements GfshCommand {
     ClusterConfigurationService service = getSharedConfiguration();
 
     if (service != null) {
-      if (isBindingAlreadyExists(jndiName))
+      Element existingBinding =
+          service.getXmlElement("cluster", "jndi-binding", "jndi-name", jndiName);
+      if (existingBinding != null) {
         throw new EntityExistsException(
             CliStrings.format("Jndi binding with jndi-name \"{0}\" already exists.", jndiName),
             ifNotExists);
+      }
       updateXml(configuration);
       persisted = true;
     }
@@ -176,12 +179,13 @@ public class CreateJndiBindingCommand implements GfshCommand {
           new CreateJndiBindingFunction(), configuration, targetMembers);
       result = buildResult(jndiCreationResult);
     } else {
-      if (persisted)
+      if (persisted) {
         result = ResultBuilder.createInfoResult(CliStrings.format(
             "No members found. Cluster configuration is updated with jndi-binding \"{0}\".",
             jndiName));
-      else
+      } else {
         result = ResultBuilder.createInfoResult("No members found.");
+      }
     }
 
     result.setCommandPersisted(persisted);
@@ -189,26 +193,6 @@ public class CreateJndiBindingCommand implements GfshCommand {
     return result;
   }
 
-  boolean isBindingAlreadyExists(String jndiName)
-      throws IOException, SAXException, ParserConfigurationException {
-
-    Configuration config = getSharedConfiguration().getConfiguration("cluster");
-
-    Document document = XmlUtils.createDocumentFromXml(config.getCacheXmlContent());
-    NodeList jndiBindings = document.getElementsByTagName("jndi-binding");
-
-    if (jndiBindings == null || jndiBindings.getLength() == 0) {
-      return false;
-    } else {
-      for (int i = 0; i < jndiBindings.getLength(); i++) {
-        Element eachBinding = (Element) jndiBindings.item(i);
-        if (eachBinding.getAttribute("jndi-name").equals(jndiName))
-          return true;
-      }
-    }
-    return false;
-  }
-
   void updateXml(JndiBindingConfiguration configuration)
       throws TransformerException, IOException, SAXException, ParserConfigurationException {
     // cluster group config should always be present
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
new file mode 100644
index 0000000..4d500b6
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommand.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.apache.geode.management.internal.cli.result.ResultBuilder.buildResult;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.TransformerException;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.DestroyJndiBindingFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.management.internal.configuration.utils.XmlUtils;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class DestroyJndiBindingCommand implements GfshCommand {
+  static final String CREATE_JNDIBINDING = "destroy jndi-binding";
+  static final String CREATE_JNDIBINDING__HELP =
+      "Destroy a jndi binding that holds the configuration for the XA datasource.";
+  static final String JNDI_NAME = "name";
+  static final String JNDI_NAME__HELP = "Name of the binding to be destroyed.";
+
+  @CliCommand(value = CREATE_JNDIBINDING, help = CREATE_JNDIBINDING__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE)
+  public Result destroyJDNIBinding(
+      @CliOption(key = JNDI_NAME, mandatory = true, help = JNDI_NAME__HELP) String jndiName)
+      throws IOException, SAXException, ParserConfigurationException, TransformerException {
+
+    Result result;
+    boolean persisted = false;
+    ClusterConfigurationService service = getSharedConfiguration();
+    if (service != null) {
+      Element existingBinding =
+          service.getXmlElement("cluster", "jndi-binding", "jndi-name", jndiName);
+      if (existingBinding == null) {
+        return ResultBuilder.createUserErrorResult(
+            CliStrings.format("Jndi binding with jndi-name \"{0}\" does not exist.", jndiName));
+      }
+      removeJndiBindingFromXml(jndiName);
+      persisted = true;
+    }
+
+    Set<DistributedMember> targetMembers = findMembers(null, null);
+    if (targetMembers.size() > 0) {
+      List<CliFunctionResult> jndiCreationResult =
+          executeAndGetFunctionResult(new DestroyJndiBindingFunction(), jndiName, targetMembers);
+      return buildResult(jndiCreationResult);
+    } else {
+      if (persisted) {
+        result = ResultBuilder.createInfoResult(CliStrings.format(
+            "No members found. Jndi-binding \"{0}\" is removed from cluster configuration.",
+            jndiName));
+      } else {
+        result = ResultBuilder.createInfoResult("No members found.");
+      }
+    }
+    result.setCommandPersisted(persisted);
+    return result;
+  }
+
+  void removeJndiBindingFromXml(String jndiName)
+      throws TransformerException, IOException, SAXException, ParserConfigurationException {
+    // cluster group config should always be present
+    Configuration config = getSharedConfiguration().getConfiguration("cluster");
+
+    Document document = XmlUtils.createDocumentFromXml(config.getCacheXmlContent());
+    NodeList jndiBindings = document.getElementsByTagName("jndi-binding");
+
+    boolean updatedXml = false;
+    if (jndiBindings != null && jndiBindings.getLength() > 0) {
+      for (int i = 0; i < jndiBindings.getLength(); i++) {
+        Element eachBinding = (Element) jndiBindings.item(i);
+        if (eachBinding.getAttribute("jndi-name").equals(jndiName)) {
+          eachBinding.getParentNode().removeChild(eachBinding);
+          updatedXml = true;
+        }
+      }
+    }
+
+    if (updatedXml) {
+      String newXml = XmlUtils.prettyXml(document.getFirstChild());
+      config.setCacheXmlContent(newXml);
+
+      getSharedConfiguration().getConfigurationRegion().put("cluster", config);
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunction.java
new file mode 100644
index 0000000..aada3c7
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunction.java
@@ -0,0 +1,44 @@
+/*
+ * 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 javax.naming.NamingException;
+
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.internal.cache.execute.InternalFunction;
+import org.apache.geode.internal.jndi.JNDIInvoker;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+
+public class DestroyJndiBindingFunction implements InternalFunction {
+
+  static final String RESULT_MESSAGE = "Jndi binding \"{0}\" destroyed on \"{1}\"";
+  static final String EXCEPTION_RESULT_MESSAGE = "Jndi binding \"{0}\" not found on \"{1}\"";
+
+  @Override
+  public void execute(FunctionContext context) {
+    ResultSender<Object> resultSender = context.getResultSender();
+    String jndiName = (String) context.getArguments();
+
+    try {
+      JNDIInvoker.unMapDatasource(jndiName);
+      resultSender.lastResult(new CliFunctionResult(context.getMemberName(), true,
+          CliStrings.format(RESULT_MESSAGE, jndiName, context.getMemberName())));
+    } catch (NamingException e) {
+      resultSender.lastResult(new CliFunctionResult(context.getMemberName(), true,
+          CliStrings.format(EXCEPTION_RESULT_MESSAGE, jndiName, context.getMemberName())));
+    }
+  }
+}
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index 16ab501..55ea33a 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -524,6 +524,7 @@ org/apache/geode/management/internal/cli/functions/DestroyAsyncEventQueueFunctio
 org/apache/geode/management/internal/cli/functions/DestroyDiskStoreFunction,true,1
 org/apache/geode/management/internal/cli/functions/DestroyDiskStoreFunctionArgs,true,2296397958405313306,id:java/lang/String,ifExists:boolean
 org/apache/geode/management/internal/cli/functions/DestroyIndexFunction,true,-868082551095130315
+org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunction,false
 org/apache/geode/management/internal/cli/functions/ExportConfigFunction,true,1
 org/apache/geode/management/internal/cli/functions/ExportDataFunction,true,1
 org/apache/geode/management/internal/cli/functions/ExportLogsFunction,true,1
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
index 59acb0e..f8401f1 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateJndiBindingCommandTest.java
@@ -106,9 +106,10 @@ public class CreateJndiBindingCommandTest {
   public void returnsErrorIfBindingAlreadyExistsAndIfUnspecified()
       throws ParserConfigurationException, SAXException, IOException {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
 
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
-    doReturn(true).when(command).isBindingAlreadyExists(any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
 
     gfsh.executeAndAssertThat(command,
         COMMAND + " --type=SIMPLE --name=name --jdbc-driver-class=driver --connection-url=url")
@@ -119,9 +120,10 @@ public class CreateJndiBindingCommandTest {
   public void skipsIfBindingAlreadyExistsAndIfSpecified()
       throws ParserConfigurationException, SAXException, IOException {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
 
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
-    doReturn(true).when(command).isBindingAlreadyExists(any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
 
     gfsh.executeAndAssertThat(command,
         COMMAND
@@ -133,9 +135,10 @@ public class CreateJndiBindingCommandTest {
   public void skipsIfBindingAlreadyExistsAndIfSpecifiedTrue()
       throws ParserConfigurationException, SAXException, IOException {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
 
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
-    doReturn(true).when(command).isBindingAlreadyExists(any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
 
     gfsh.executeAndAssertThat(command,
         COMMAND
@@ -147,9 +150,10 @@ public class CreateJndiBindingCommandTest {
   public void returnsErrorIfBindingAlreadyExistsAndIfSpecifiedFalse()
       throws ParserConfigurationException, SAXException, IOException {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
 
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
-    doReturn(true).when(command).isBindingAlreadyExists(any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
 
     gfsh.executeAndAssertThat(command,
         COMMAND
@@ -158,66 +162,6 @@ public class CreateJndiBindingCommandTest {
   }
 
   @Test
-  public void ifBindingAlreadyExistsShouldReturnTrueIfABindingExistsWithJndiName()
-      throws ParserConfigurationException, SAXException, IOException, TransformerException {
-
-    Configuration clusterConfig = new Configuration("cluster");
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
-    Region configRegion = mock(Region.class);
-
-    doReturn(configRegion).when(clusterConfigService).getConfigurationRegion();
-    doReturn(null).when(configRegion).put(any(), any());
-    doReturn(clusterConfig).when(clusterConfigService).getConfiguration("cluster");
-    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
-    doReturn(clusterConfigService).when(command).getSharedConfiguration();
-
-    // update cluster config
-    JndiBindingConfiguration jndi = new JndiBindingConfiguration();
-    jndi.setJndiName("jndi1");
-    jndi.setType(JndiBindingConfiguration.DATASOURCE_TYPE.SIMPLE);
-    command.updateXml(jndi);
-
-    assertThat(command.isBindingAlreadyExists("jndi1")).isTrue();
-  }
-
-  @Test
-  public void ifBindingAlreadyExistsShouldReturnFalseIfNoBindingsTagExists()
-      throws ParserConfigurationException, SAXException, IOException {
-
-    Configuration clusterConfig = new Configuration("cluster");
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
-
-    doReturn(clusterConfig).when(clusterConfigService).getConfiguration("cluster");
-    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
-    doReturn(clusterConfigService).when(command).getSharedConfiguration();
-
-    assertThat(command.isBindingAlreadyExists("somename")).isFalse();
-  }
-
-  @Test
-  public void ifBindingAlreadyExistsShouldReturnFalseIfNoBindingExists()
-      throws ParserConfigurationException, SAXException, IOException, TransformerException {
-
-    Configuration clusterConfig = new Configuration("cluster");
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
-    Region configRegion = mock(Region.class);
-
-    doReturn(configRegion).when(clusterConfigService).getConfigurationRegion();
-    doReturn(null).when(configRegion).put(any(), any());
-    doReturn(clusterConfig).when(clusterConfigService).getConfiguration("cluster");
-    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
-    doReturn(clusterConfigService).when(command).getSharedConfiguration();
-
-    // update cluster config
-    JndiBindingConfiguration jndi = new JndiBindingConfiguration();
-    jndi.setJndiName("jndi1");
-    jndi.setType(JndiBindingConfiguration.DATASOURCE_TYPE.SIMPLE);
-    command.updateXml(jndi);
-
-    assertThat(command.isBindingAlreadyExists("jndi2")).isFalse();
-  }
-
-  @Test
   public void updateXmlShouldClusterConfigurationWithJndiConfiguration()
       throws IOException, ParserConfigurationException, SAXException, TransformerException {
     Configuration clusterConfig = new Configuration("cluster");
@@ -297,7 +241,7 @@ public class CreateJndiBindingCommandTest {
       throws IOException, ParserConfigurationException, SAXException, TransformerException {
 
     doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
-    doReturn(false).when(command).isBindingAlreadyExists(any());
+    doReturn(null).when(command).getSharedConfiguration();
     doNothing().when(command).updateXml(any());
 
     gfsh.executeAndAssertThat(command,
@@ -311,7 +255,7 @@ public class CreateJndiBindingCommandTest {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
 
     doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
-    doReturn(false).when(command).isBindingAlreadyExists(any());
+    doReturn(null).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
     doNothing().when(command).updateXml(any());
 
@@ -335,10 +279,8 @@ public class CreateJndiBindingCommandTest {
         "Tried creating jndi binding \"name\" on \"server1\"");
     List<CliFunctionResult> results = new ArrayList<>();
     results.add(result);
-    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
 
     doReturn(members).when(command).findMembers(any(), any());
-    doReturn(false).when(command).isBindingAlreadyExists(any());
     doReturn(null).when(command).getSharedConfiguration();
     doReturn(results).when(command).executeAndGetFunctionResult(any(), any(), any());
 
@@ -380,7 +322,7 @@ public class CreateJndiBindingCommandTest {
     ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
 
     doReturn(members).when(command).findMembers(any(), any());
-    doReturn(false).when(command).isBindingAlreadyExists(any());
+    doReturn(null).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
     doReturn(clusterConfigService).when(command).getSharedConfiguration();
     doNothing().when(command).updateXml(any());
     doReturn(results).when(command).executeAndGetFunctionResult(any(), any(), any());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java
new file mode 100644
index 0000000..63f8c37
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandDUnitTest.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.internal.jndi.JNDIInvoker;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.management.internal.configuration.utils.XmlUtils;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.VMProvider;
+
+@Category(DistributedTest.class)
+public class DestroyJndiBindingCommandDUnitTest {
+
+  private static MemberVM locator, server1, server2;
+
+  @ClassRule
+  public static ClusterStartupRule cluster = new ClusterStartupRule();
+
+  @ClassRule
+  public static GfshCommandRule gfsh = new GfshCommandRule();
+
+  @BeforeClass
+  public static void before() throws Exception {
+    locator = cluster.startLocatorVM(0);
+    server1 = cluster.startServerVM(1, locator.getPort());
+    server2 = cluster.startServerVM(2, locator.getPort());
+
+    gfsh.connectAndVerify(locator);
+
+    // create the binding
+    gfsh.execute(
+        "create jndi-binding --name=jndi1 --type=SIMPLE --jdbc-driver-class=org.apache.derby.jdbc.EmbeddedDriver --connection-url=\"jdbc:derby:newDB;create=true\"");
+  }
+
+  @Test
+  public void testDestroyJndiBinding() throws Exception {
+    // assert that there is a datasource
+    VMProvider.invokeInEveryMember(
+        () -> assertThat(JNDIInvoker.getBindingNamesRecursively(JNDIInvoker.getJNDIContext()))
+            .containsKey("jndi1").containsValue(
+                "org.apache.geode.internal.datasource.GemFireBasicDataSource"));
+
+    gfsh.executeAndAssertThat("destroy jndi-binding --name=jndi1").statusIsSuccess()
+        .tableHasColumnOnlyWithValues("Member", "server-1", "server-2");
+
+    // verify cluster config is updated
+    locator.invoke(() -> {
+      ClusterConfigurationService ccService =
+          ClusterStartupRule.getLocator().getSharedConfiguration();
+      Configuration configuration = ccService.getConfiguration("cluster");
+      Document document = XmlUtils.createDocumentFromXml(configuration.getCacheXmlContent());
+      NodeList jndiBindings = document.getElementsByTagName("jndi-binding");
+
+      assertThat(jndiBindings.getLength()).isEqualTo(0);
+
+      boolean found = false;
+      for (int i = 0; i < jndiBindings.getLength(); i++) {
+        Element eachBinding = (Element) jndiBindings.item(i);
+        if (eachBinding.getAttribute("jndi-name").equals("jndi1")) {
+          found = true;
+          break;
+        }
+      }
+      assertThat(found).isFalse();
+    });
+
+    // verify datasource exists
+    VMProvider.invokeInEveryMember(
+        () -> assertThat(JNDIInvoker.getNoOfAvailableDataSources()).isEqualTo(0));
+
+    // bounce server1 and assert that there is still no datasource received from cluster config
+    server1.stopVM(false);
+    server1 = cluster.startServerVM(1, locator.getPort());
+
+    // verify no datasource from cluster config
+    server1.invoke(() -> {
+      assertThat(JNDIInvoker.getNoOfAvailableDataSources()).isEqualTo(0);
+    });
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
new file mode 100644
index 0000000..0538e62
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DestroyJndiBindingCommandTest.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.TransformerException;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.xml.sax.SAXException;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.ClusterConfigurationService;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.datasource.ConfigProperty;
+import org.apache.geode.management.internal.cli.GfshParseResult;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.CreateJndiBindingFunction;
+import org.apache.geode.management.internal.cli.functions.DestroyJndiBindingFunction;
+import org.apache.geode.management.internal.cli.functions.JndiBindingConfiguration;
+import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.geode.management.internal.configuration.utils.XmlUtils;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.apache.geode.test.junit.rules.GfshParserRule;
+
+@Category(UnitTest.class)
+public class DestroyJndiBindingCommandTest {
+
+  @ClassRule
+  public static GfshParserRule gfsh = new GfshParserRule();
+
+  private DestroyJndiBindingCommand command;
+  private InternalCache cache;
+
+  private static String COMMAND = "destroy jndi-binding ";
+
+  @Before
+  public void setUp() throws Exception {
+    cache = mock(InternalCache.class);
+    command = spy(DestroyJndiBindingCommand.class);
+    doReturn(cache).when(command).getCache();
+  }
+
+  @Test
+  public void missingMandatory() {
+    gfsh.executeAndAssertThat(command, COMMAND).statusIsError().containsOutput("Invalid command");
+  }
+
+  @Test
+  public void returnsErrorIfNoBindingExistsWithGivenName()
+      throws ParserConfigurationException, SAXException, IOException {
+    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    doReturn(clusterConfigService).when(command).getSharedConfiguration();
+    doReturn(null).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
+    gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsError()
+        .containsOutput("does not exist.");
+  }
+
+  @Test
+  public void removeJndiBindingFromXmlShouldRemoveBindingFromClusterConfiguration()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Region configRegion = mock(Region.class);
+
+    Configuration clusterConfig = new Configuration("cluster");
+    String xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>"
+        + "<cache xmlns=\"http://geode.apache.org/schema/cache\" "
+        + "xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\" "
+        + "copy-on-read=\"false\" is-server=\"false\" lock-lease=\"120\" "
+        + "lock-timeout=\"60\" search-timeout=\"300\" version=\"1.0\" "
+        + "xsi:schemaLocation=\"http://geode.apache.org/schema/cache "
+        + "http://geode.apache.org/schema/cache/cache-1.0.xsd\">" + "<jndi-bindings>"
+        + "<jndi-binding jndi-name=\"jndi1\" type=\"SimpleDataSource\"/>"
+        + "<jndi-binding jndi-name=\"jndi2\" type=\"SimpleDataSource\"/>"
+        + "</jndi-bindings></cache>";
+    clusterConfig.setCacheXmlContent(xml);
+
+    doReturn(configRegion).when(clusterConfigService).getConfigurationRegion();
+    doReturn(null).when(configRegion).put(any(), any());
+    doReturn(clusterConfig).when(clusterConfigService).getConfiguration("cluster");
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    doReturn(clusterConfigService).when(command).getSharedConfiguration();
+
+    command.removeJndiBindingFromXml("jndi1");
+    assertThat(clusterConfig.getCacheXmlContent()).doesNotContain("jndi1");
+    assertThat(clusterConfig.getCacheXmlContent()).contains("jndi2");
+    verify(configRegion, times(1)).put("cluster", clusterConfig);
+  }
+
+  @Test
+  public void removeJndiBindingFromXmlIsNoOpWhenNoBindingFoundInClusterConfiguration()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Region configRegion = mock(Region.class);
+    Configuration clusterConfig = new Configuration("cluster");
+
+    doReturn(configRegion).when(clusterConfigService).getConfigurationRegion();
+    doReturn(null).when(configRegion).put(any(), any());
+    doReturn(clusterConfig).when(clusterConfigService).getConfiguration("cluster");
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    doReturn(clusterConfigService).when(command).getSharedConfiguration();
+
+    command.removeJndiBindingFromXml("jndi1");
+    verify(configRegion, times(0)).put("cluster", clusterConfig);
+  }
+
+  @Test
+  public void whenNoMembersFoundAndNoClusterConfigServiceRunningThenError()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    doReturn(null).when(command).getSharedConfiguration();
+    doNothing().when(command).removeJndiBindingFromXml(any());
+
+    gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
+        .containsOutput("No members found").hasFailToPersistError();
+  }
+
+  @Test
+  public void whenNoMembersFoundAndClusterConfigRunningThenUpdateClusterConfig()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
+
+    doReturn(Collections.emptySet()).when(command).findMembers(any(), any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
+    doReturn(clusterConfigService).when(command).getSharedConfiguration();
+    doNothing().when(command).removeJndiBindingFromXml(any());
+
+    gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
+        .containsOutput(
+            "No members found. Jndi-binding \\\"name\\\" is removed from cluster configuration.")
+        .hasNoFailToPersistError();
+
+    verify(command, times(1)).removeJndiBindingFromXml(any());
+  }
+
+  @Test
+  public void whenMembersFoundAndNoClusterConfigRunningThenOnlyInvokeFunction()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    Set<DistributedMember> members = new HashSet<>();
+    members.add(mock(DistributedMember.class));
+
+    CliFunctionResult result =
+        new CliFunctionResult("server1", true, "Jndi binding \"name\" destroyed on \"server1\"");
+    List<CliFunctionResult> results = new ArrayList<>();
+    results.add(result);
+
+    doReturn(members).when(command).findMembers(any(), any());
+    doReturn(null).when(command).getSharedConfiguration();
+    doReturn(results).when(command).executeAndGetFunctionResult(any(), any(), any());
+
+    gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
+        .tableHasColumnOnlyWithValues("Member", "server1")
+        .tableHasColumnOnlyWithValues("Status", "Jndi binding \"name\" destroyed on \"server1\"");
+
+    verify(command, times(0)).removeJndiBindingFromXml(any());
+
+    ArgumentCaptor<DestroyJndiBindingFunction> function =
+        ArgumentCaptor.forClass(DestroyJndiBindingFunction.class);
+    ArgumentCaptor<String> jndiName = ArgumentCaptor.forClass(String.class);
+    ArgumentCaptor<Set<DistributedMember>> targetMembers = ArgumentCaptor.forClass(Set.class);
+    verify(command, times(1)).executeAndGetFunctionResult(function.capture(), jndiName.capture(),
+        targetMembers.capture());
+
+    assertThat(function.getValue()).isInstanceOf(DestroyJndiBindingFunction.class);
+    assertThat(jndiName.getValue()).isNotNull();
+    assertThat(jndiName.getValue()).isEqualTo("name");
+    assertThat(targetMembers.getValue()).isEqualTo(members);
+  }
+
+  @Test
+  public void whenMembersFoundAndClusterConfigRunningThenUpdateClusterConfigAndInvokeFunction()
+      throws IOException, ParserConfigurationException, SAXException, TransformerException {
+    Set<DistributedMember> members = new HashSet<>();
+    members.add(mock(DistributedMember.class));
+
+    CliFunctionResult result =
+        new CliFunctionResult("server1", true, "Jndi binding \"name\" destroyed on \"server1\"");
+    List<CliFunctionResult> results = new ArrayList<>();
+    results.add(result);
+    ClusterConfigurationService clusterConfigService = mock(ClusterConfigurationService.class);
+    Element existingBinding = mock(Element.class);
+
+    doReturn(members).when(command).findMembers(any(), any());
+    doReturn(existingBinding).when(clusterConfigService).getXmlElement(any(), any(), any(), any());
+    doReturn(clusterConfigService).when(command).getSharedConfiguration();
+    doNothing().when(command).removeJndiBindingFromXml(any());
+    doReturn(results).when(command).executeAndGetFunctionResult(any(), any(), any());
+
+    gfsh.executeAndAssertThat(command, COMMAND + " --name=name").statusIsSuccess()
+        .tableHasColumnOnlyWithValues("Member", "server1")
+        .tableHasColumnOnlyWithValues("Status", "Jndi binding \"name\" destroyed on \"server1\"");
+
+    verify(command, times(1)).removeJndiBindingFromXml(any());
+
+    ArgumentCaptor<CreateJndiBindingFunction> function =
+        ArgumentCaptor.forClass(CreateJndiBindingFunction.class);
+    ArgumentCaptor<String> jndiName = ArgumentCaptor.forClass(String.class);
+    ArgumentCaptor<Set<DistributedMember>> targetMembers = ArgumentCaptor.forClass(Set.class);
+    verify(command, times(1)).executeAndGetFunctionResult(function.capture(), jndiName.capture(),
+        targetMembers.capture());
+
+    assertThat(function.getValue()).isInstanceOf(DestroyJndiBindingFunction.class);
+    assertThat(jndiName.getValue()).isNotNull();
+    assertThat(jndiName.getValue()).isEqualTo("name");
+    assertThat(targetMembers.getValue()).isEqualTo(members);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunctionTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunctionTest.java
new file mode 100644
index 0000000..42f369f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/DestroyJndiBindingFunctionTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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 static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.logging.Level;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.jndi.JNDIInvoker;
+import org.apache.geode.internal.logging.LocalLogWriter;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class DestroyJndiBindingFunctionTest {
+
+  private DestroyJndiBindingFunction destroyJndiBindingFunction;
+  private FunctionContext context;
+  private DistributedSystem distributedSystem;
+  private ResultSender resultSender;
+  private ArgumentCaptor<CliFunctionResult> resultCaptor;
+
+  @Before
+  public void setup() {
+    destroyJndiBindingFunction = spy(new DestroyJndiBindingFunction());
+    context = mock(FunctionContext.class);
+
+    distributedSystem = mock(DistributedSystem.class);
+    resultSender = mock(ResultSender.class);
+    resultCaptor = ArgumentCaptor.forClass(CliFunctionResult.class);
+
+    when(distributedSystem.getLogWriter()).thenReturn(new LocalLogWriter(Level.FINE.intValue()));
+
+    JNDIInvoker.mapTransactions(distributedSystem);
+
+    JndiBindingConfiguration config = new JndiBindingConfiguration();
+    config.setJndiName("jndi1");
+    config.setType(JndiBindingConfiguration.DATASOURCE_TYPE.SIMPLE);
+    config.setJdbcDriver("org.apache.derby.jdbc.EmbeddedDriver");
+    config.setConnectionUrl("jdbc:derby:newDB;create=true");
+    JNDIInvoker.mapDatasource(config.getParamsAsMap(), config.getDatasourceConfigurations());
+  }
+
+  @Test
+  public void destroyJndiBindingIsSuccessfulWhenBindingFound() throws Exception {
+    when(context.getArguments()).thenReturn("jndi1");
+    when(context.getMemberName()).thenReturn("server-1");
+    when(context.getResultSender()).thenReturn(resultSender);
+
+    destroyJndiBindingFunction.execute(context);
+
+    verify(resultSender).lastResult(resultCaptor.capture());
+    CliFunctionResult result = resultCaptor.getValue();
+
+    assertThat(result.isSuccessful()).isTrue();
+    assertThat(result.getMessage()).contains("Jndi binding \"jndi1\" destroyed on \"server-1\"");
+  }
+
+  @Test
+  public void destroyJndiBindingIsSuccessfulWhenNoBindingFound() throws Exception {
+    when(context.getArguments()).thenReturn("somectx/unknownjndi");
+    when(context.getResultSender()).thenReturn(resultSender);
+
+    destroyJndiBindingFunction.execute(context);
+
+    verify(resultSender).lastResult(resultCaptor.capture());
+    CliFunctionResult result = resultCaptor.getValue();
+
+    assertThat(result.isSuccessful()).isTrue();
+    assertThat(result.getMessage()).contains("not found");
+  }
+}

-- 
To stop receiving notification emails like this one, please contact
sai_boorlagadda@apache.org.