You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ni...@apache.org on 2020/03/27 12:19:55 UTC

[ignite] branch master updated: IGNITE-12812: Cancel transaction command. (#7568)

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

nizhikov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 3f0f818  IGNITE-12812: Cancel transaction command. (#7568)
3f0f818 is described below

commit 3f0f818bd6510ffcde9de7545d4f88805d23ba36
Author: Nikolay <ni...@apache.org>
AuthorDate: Fri Mar 27 15:19:35 2020 +0300

    IGNITE-12812: Cancel transaction command. (#7568)
---
 .../ignite/internal/TransactionsMXBeanImpl.java    | 18 +++++--
 .../internal/commandline/query/KillCommand.java    | 34 ++++++++----
 .../internal/commandline/query/KillSubcommand.java |  5 ++
 .../org/apache/ignite/internal/sql/SqlParser.java  |  4 ++
 .../sql/command/SqlKillTransactionCommand.java     | 63 ++++++++++++++++++++++
 .../apache/ignite/mxbean/TransactionsMXBean.java   | 12 +++++
 ...ridCommandHandlerClusterByClassTest_help.output |  6 +++
 ...andHandlerClusterByClassWithSSLTest_help.output |  6 +++
 .../processors/query/h2/CommandProcessor.java      | 13 +++++
 .../internal/processors/query/h2/QueryParser.java  |  6 ++-
 .../ignite/util/KillCommandsCommandShTest.java     | 25 +++++++++
 .../apache/ignite/util/KillCommandsMXBeanTest.java | 27 ++++++++++
 .../apache/ignite/util/KillCommandsSQLTest.java    | 23 ++++++++
 .../org/apache/ignite/util/KillCommandsTests.java  | 42 +++++++++++++++
 14 files changed, 268 insertions(+), 16 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java
index e9dd008..5ca0a90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorTaskArgument;
 import org.apache.ignite.internal.visor.tx.VisorTxInfo;
@@ -40,13 +41,13 @@ import org.apache.ignite.mxbean.TransactionsMXBean;
  * TransactionsMXBean implementation.
  */
 public class TransactionsMXBeanImpl implements TransactionsMXBean {
-    /** */
-    private final GridKernalContextImpl ctx;
+    /** Kernal context. */
+    private final GridKernalContext ctx;
 
     /**
      * @param ctx Context.
      */
-    public TransactionsMXBeanImpl(GridKernalContextImpl ctx) {
+    public TransactionsMXBeanImpl(GridKernalContext ctx) {
         this.ctx = ctx;
     }
 
@@ -117,6 +118,17 @@ public class TransactionsMXBeanImpl implements TransactionsMXBean {
     }
 
     /** {@inheritDoc} */
+    @Override public void cancel(String xid) {
+        A.notNull(xid, "xid");
+
+        IgniteCompute compute = ctx.cluster().get().compute();
+
+        compute.execute(new VisorTxTask(),
+            new VisorTaskArgument<>(ctx.localNodeId(), new VisorTxTaskArg(VisorTxOperation.KILL,
+                1, null, null, null, null, null, xid, null, null, null), false));
+    }
+
+    /** {@inheritDoc} */
     @Override public long getTxTimeoutOnPartitionMapExchange() {
         return ctx.config().getTransactionConfiguration().getTxTimeoutOnPartitionMapExchange();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
index 0d5e577..d600fd3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillCommand.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.commandline.query;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.logging.Logger;
 import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.client.GridClientConfiguration;
@@ -30,13 +28,19 @@ import org.apache.ignite.internal.visor.service.VisorCancelServiceTaskArg;
 import org.apache.ignite.mxbean.ServiceMXBean;
 import org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionTask;
 import org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionTaskArg;
+import org.apache.ignite.internal.visor.tx.VisorTxOperation;
+import org.apache.ignite.internal.visor.tx.VisorTxTask;
+import org.apache.ignite.internal.visor.tx.VisorTxTaskArg;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.mxbean.ComputeMXBean;
+import org.apache.ignite.mxbean.TransactionsMXBean;
 
+import static java.util.Collections.singletonMap;
 import static org.apache.ignite.internal.commandline.CommandList.KILL;
 import static org.apache.ignite.internal.commandline.TaskExecutor.executeTaskByNameOnNode;
 import static org.apache.ignite.internal.commandline.query.KillSubcommand.SERVICE;
 import static org.apache.ignite.internal.commandline.query.KillSubcommand.COMPUTE;
+import static org.apache.ignite.internal.commandline.query.KillSubcommand.TRANSACTION;
 
 /**
  * control.sh kill command.
@@ -44,6 +48,7 @@ import static org.apache.ignite.internal.commandline.query.KillSubcommand.COMPUT
  * @see KillSubcommand
  * @see ServiceMXBean
  * @see ComputeMXBean
+ * @see TransactionsMXBean
  */
 public class KillCommand implements Command<Object> {
     /** Command argument. */
@@ -103,6 +108,16 @@ public class KillCommand implements Command<Object> {
 
                 break;
 
+            case TRANSACTION:
+                String xid = argIter.nextArg("Expected transaction id.");
+
+                taskArgs = new VisorTxTaskArg(VisorTxOperation.KILL, null, null, null, null, null, null, xid, null,
+                    null, null);
+
+                taskName = VisorTxTask.class.getName();
+
+                break;
+
             default:
                 throw new IllegalArgumentException("Unknown kill subcommand: " + cmd);
         }
@@ -110,17 +125,14 @@ public class KillCommand implements Command<Object> {
 
     /** {@inheritDoc} */
     @Override public void printUsage(Logger log) {
-        Map<String, String> params = new HashMap<>();
-
-        params.put("session_id", "Session identifier.");
-
-        Command.usage(log, "Kill compute task by session id:", KILL, params, COMPUTE.toString(),
-            "session_id");
+        Command.usage(log, "Kill compute task by session id:", KILL, singletonMap("session_id", "Session identifier."),
+            COMPUTE.toString(), "session_id");
 
-        params.clear();
-        params.put("name", "Service name.");
+        Command.usage(log, "Kill service by name:", KILL, singletonMap("name", "Service name."),
+            SERVICE.toString(), "name");
 
-        Command.usage(log, "Kill service by name:", KILL, params, SERVICE.toString(), "name");
+        Command.usage(log, "Kill transaction by xid:", KILL, singletonMap("xid", "Transaction identifier."),
+            TRANSACTION.toString(), "xid");
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillSubcommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillSubcommand.java
index 9ad7d7af..d07a27d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillSubcommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/query/KillSubcommand.java
@@ -19,18 +19,23 @@ package org.apache.ignite.internal.commandline.query;
 
 import org.apache.ignite.mxbean.ComputeMXBean;
 import org.apache.ignite.mxbean.ServiceMXBean;
+import org.apache.ignite.mxbean.TransactionsMXBean;
 
 /**
  * Subcommands of the kill command.
  *
  * @see KillCommand
  * @see ComputeMXBean
+ * @see TransactionsMXBean
  * @see ServiceMXBean
  */
 public enum KillSubcommand {
     /** Kill compute task. */
     COMPUTE,
 
+    /** Kill transaction. */
+    TRANSACTION,
+
     /** Kill service. */
     SERVICE
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
index aa7a24a..fdfa296 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
 import org.apache.ignite.internal.sql.command.SqlKillComputeTaskCommand;
 import org.apache.ignite.internal.sql.command.SqlKillQueryCommand;
 import org.apache.ignite.internal.sql.command.SqlKillServiceCommand;
+import org.apache.ignite.internal.sql.command.SqlKillTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
 import org.jetbrains.annotations.Nullable;
@@ -299,6 +300,9 @@ public class SqlParser {
 
                 case SERVICE:
                     return new SqlKillServiceCommand().parse(lex);
+
+                case TRANSACTION:
+                    return new SqlKillTransactionCommand().parse(lex);
             }
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlKillTransactionCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlKillTransactionCommand.java
new file mode 100644
index 0000000..5135c51
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlKillTransactionCommand.java
@@ -0,0 +1,63 @@
+/*
+ * 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.ignite.internal.sql.command;
+
+import org.apache.ignite.internal.sql.SqlLexer;
+import org.apache.ignite.internal.sql.SqlLexerTokenType;
+import org.apache.ignite.internal.sql.SqlParserUtils;
+import org.apache.ignite.mxbean.TransactionsMXBean;
+import org.apache.ignite.spi.systemview.view.TransactionView;
+
+/**
+ * KILL TRANSACTION command.
+ *
+ * @see TransactionsMXBean#cancel(String)
+ * @see TransactionView#xid()
+ */
+public class SqlKillTransactionCommand implements SqlCommand {
+    /** Transaction xid. */
+    private String xid;
+
+    /** {@inheritDoc} */
+    @Override public SqlCommand parse(SqlLexer lex) {
+        if (lex.shift()) {
+            if (lex.tokenType() == SqlLexerTokenType.STRING) {
+                xid = lex.token();
+
+                return this;
+            }
+        }
+
+        throw SqlParserUtils.error(lex, "Expected transaction xid.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String schemaName() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void schemaName(String schemaName) {
+        // No-op.
+    }
+
+    /** @return Xid. */
+    public String getXid() {
+        return xid;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java
index f7f8831..fca905c 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.mxbean;
 
 import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.spi.systemview.view.TransactionView;
 
 /**
  * Transactions MXBean interface.
@@ -61,6 +62,17 @@ public interface TransactionsMXBean {
     );
 
     /**
+     * Kills transactions by the xid.
+     *
+     * @param xid Transaction xid.
+     * @see TransactionView#xid()
+     */
+    @MXBeanDescription("Kills transactions by the xid.")
+    public void cancel(
+        @MXBeanParameter(name = "xid", description = "Transaction XID.") String xid
+    );
+
+    /**
      * Gets transaction timeout on partition map exchange.
      * <p>
      * @see TransactionConfiguration#getTxTimeoutOnPartitionMapExchange
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
index e83571c..0f91993 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
@@ -80,6 +80,12 @@ This utility can do the following commands:
     Parameters:
       name  - Service name.
 
+  Kill transaction by xid:
+    control.(sh|bat) --kill TRANSACTION xid
+
+    Parameters:
+      xid  - Transaction identifier.
+
 By default commands affecting the cluster require interactive confirmation.
 Use --yes option to disable it.
 
diff --git a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
index e83571c..0f91993 100644
--- a/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
+++ b/modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassWithSSLTest_help.output
@@ -80,6 +80,12 @@ This utility can do the following commands:
     Parameters:
       name  - Service name.
 
+  Kill transaction by xid:
+    control.(sh|bat) --kill TRANSACTION xid
+
+    Parameters:
+      xid  - Transaction identifier.
+
 By default commands affecting the cluster require interactive confirmation.
 Use --yes option to disable it.
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
index 676b0b4..a24d307 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.ServiceMXBeanImpl;
+import org.apache.ignite.internal.TransactionsMXBeanImpl;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters;
@@ -101,6 +102,7 @@ import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
 import org.apache.ignite.internal.sql.command.SqlIndexColumn;
 import org.apache.ignite.internal.sql.command.SqlKillComputeTaskCommand;
 import org.apache.ignite.internal.sql.command.SqlKillQueryCommand;
+import org.apache.ignite.internal.sql.command.SqlKillTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlKillServiceCommand;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
@@ -415,6 +417,8 @@ public class CommandProcessor {
                 processKillQueryCommand((SqlKillQueryCommand) cmdNative);
             else if (cmdNative instanceof SqlKillComputeTaskCommand)
                 processKillComputeTaskCommand((SqlKillComputeTaskCommand) cmdNative);
+            else if (cmdNative instanceof SqlKillTransactionCommand)
+                processKillTxCommand((SqlKillTransactionCommand) cmdNative);
             else if (cmdNative instanceof SqlKillServiceCommand)
                 processKillServiceTaskCommand((SqlKillServiceCommand) cmdNative);
             else
@@ -505,6 +509,15 @@ public class CommandProcessor {
     }
 
     /**
+     * Process kill transaction command.
+     *
+     * @param command Command.
+     */
+    private void processKillTxCommand(SqlKillTransactionCommand command) {
+        new TransactionsMXBeanImpl(ctx).cancel(command.getXid());
+    }
+
+    /**
      * Process kill service command.
      *
      * @param cmd Command.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
index 1a391b8..6d9c436 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
@@ -72,6 +72,7 @@ import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
 import org.apache.ignite.internal.sql.command.SqlKillComputeTaskCommand;
 import org.apache.ignite.internal.sql.command.SqlKillQueryCommand;
 import org.apache.ignite.internal.sql.command.SqlKillServiceCommand;
+import org.apache.ignite.internal.sql.command.SqlKillTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
@@ -92,7 +93,7 @@ public class QueryParser {
     /** A pattern for commands having internal implementation in Ignite. */
     private static final Pattern INTERNAL_CMD_RE = Pattern.compile(
         "^(create|drop)\\s+index|^alter\\s+table|^copy|^set|^begin|^commit|^rollback|^(create|alter|drop)\\s+user" +
-            "|^kill\\s+(query|compute|service)|show|help|grant|revoke",
+            "|^kill\\s+(query|compute|service|transaction)|show|help|grant|revoke",
         Pattern.CASE_INSENSITIVE);
 
     /** Indexing. */
@@ -268,7 +269,8 @@ public class QueryParser {
                 || nativeCmd instanceof SqlDropUserCommand
                 || nativeCmd instanceof SqlKillQueryCommand
                 || nativeCmd instanceof SqlKillComputeTaskCommand
-                || nativeCmd instanceof SqlKillServiceCommand)
+                || nativeCmd instanceof SqlKillServiceCommand
+                || nativeCmd instanceof SqlKillTransactionCommand)
             )
                 return null;
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
index 2489394..a0f52f7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsCommandShTest.java
@@ -19,12 +19,15 @@ package org.apache.ignite.util;
 
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteUuid;
 import org.junit.Test;
 
 import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelComputeTask;
+import static org.apache.ignite.util.KillCommandsTests.doTestCancelTx;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelService;
 
 /** Tests cancel of user created entities via control.sh. */
@@ -41,6 +44,10 @@ public class KillCommandsCommandShTest extends GridCommandHandlerClusterByClassA
         for (int i = 0; i < SERVER_NODE_CNT; i++)
             srvs.add(grid(i));
 
+        client.getOrCreateCache(
+            new CacheConfiguration<>(DEFAULT_CACHE_NAME).setIndexedTypes(Integer.class, Integer.class)
+                .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
+
         awaitPartitionMapExchange();
     }
 
@@ -59,6 +66,16 @@ public class KillCommandsCommandShTest extends GridCommandHandlerClusterByClassA
         });
     }
 
+    /** */
+    @Test
+    public void testCancelTx() {
+        doTestCancelTx(client, srvs, xid -> {
+            int res = execute("--kill", "transaction", xid);
+
+            assertEquals(EXIT_CODE_OK, res);
+        });
+    }
+
     /** @throws Exception If failed. */
     @Test
     public void testCancelService() throws Exception {
@@ -84,4 +101,12 @@ public class KillCommandsCommandShTest extends GridCommandHandlerClusterByClassA
 
         assertEquals(EXIT_CODE_OK, res);
     }
+
+    /** */
+    @Test
+    public void testCancelUnknownTx() {
+        int res = execute("--kill", "transaction", "unknown");
+
+        assertEquals(EXIT_CODE_OK, res);
+    }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsMXBeanTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsMXBeanTest.java
index 5ca33fa..000e3c2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsMXBeanTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsMXBeanTest.java
@@ -19,18 +19,23 @@ package org.apache.ignite.util;
 
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.ComputeMXBeanImpl;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TransactionsMXBeanImpl;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.mxbean.ComputeMXBean;
 import org.apache.ignite.internal.ServiceMXBeanImpl;
 import org.apache.ignite.mxbean.ServiceMXBean;
+import org.apache.ignite.mxbean.TransactionsMXBean;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
 import static org.apache.ignite.cluster.ClusterState.ACTIVE;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelComputeTask;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelService;
+import static org.apache.ignite.util.KillCommandsTests.doTestCancelTx;
 
 /** Tests cancel of user created entities via JMX. */
 public class KillCommandsMXBeanTest extends GridCommonAbstractTest {
@@ -47,6 +52,9 @@ public class KillCommandsMXBeanTest extends GridCommonAbstractTest {
     private static IgniteEx killCli;
 
     /** */
+    private static TransactionsMXBean txMBean;
+
+    /** */
     private static ComputeMXBean computeMBean;
 
     /** */
@@ -66,6 +74,13 @@ public class KillCommandsMXBeanTest extends GridCommonAbstractTest {
 
         srvs.get(0).cluster().state(ACTIVE);
 
+        startCli.getOrCreateCache(
+            new CacheConfiguration<>(DEFAULT_CACHE_NAME).setIndexedTypes(Integer.class, Integer.class)
+                .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
+
+        txMBean = getMxBean(killCli.name(), "Transactions",
+            TransactionsMXBeanImpl.class.getSimpleName(), TransactionsMXBean.class);
+
         computeMBean = getMxBean(killCli.name(), "Compute",
             ComputeMXBeanImpl.class.getSimpleName(), ComputeMXBean.class);
 
@@ -87,12 +102,24 @@ public class KillCommandsMXBeanTest extends GridCommonAbstractTest {
 
     /** */
     @Test
+    public void testCancelTx() {
+        doTestCancelTx(startCli, srvs, xid -> txMBean.cancel(xid));
+    }
+
+    /** */
+    @Test
     public void testCancelUnknownComputeTask() {
         computeMBean.cancel(IgniteUuid.randomUuid().toString());
     }
 
     /** */
     @Test
+    public void testCancelUnknownTx() {
+        txMBean.cancel("unknown");
+    }
+
+    /** */
+    @Test
     public void testCancelUnknownService() {
         svcMxBean.cancel("unknown");
     }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsSQLTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsSQLTest.java
index 2a5cc8e..f23f522 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsSQLTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsSQLTest.java
@@ -20,7 +20,9 @@ package org.apache.ignite.util;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -31,8 +33,10 @@ import static org.apache.ignite.internal.processors.cache.index.AbstractSchemaSe
 import static org.apache.ignite.internal.sql.SqlKeyword.COMPUTE;
 import static org.apache.ignite.internal.sql.SqlKeyword.KILL;
 import static org.apache.ignite.internal.sql.SqlKeyword.SERVICE;
+import static org.apache.ignite.internal.sql.SqlKeyword.TRANSACTION;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelComputeTask;
 import static org.apache.ignite.util.KillCommandsTests.doTestCancelService;
+import static org.apache.ignite.util.KillCommandsTests.doTestCancelTx;
 
 /** Tests cancel of user created entities via SQL. */
 public class KillCommandsSQLTest extends GridCommonAbstractTest {
@@ -46,6 +50,9 @@ public class KillCommandsSQLTest extends GridCommonAbstractTest {
     public static final String KILL_SVC_QRY = KILL + " " + SERVICE;
 
     /** */
+    public static final String KILL_TX_QRY = KILL + " " + TRANSACTION;
+
+    /** */
     private static List<IgniteEx> srvs;
 
     /** Client that starts tasks. */
@@ -67,6 +74,10 @@ public class KillCommandsSQLTest extends GridCommonAbstractTest {
         killCli = startClientGrid("killClient");
 
         srvs.get(0).cluster().state(ACTIVE);
+
+        startCli.getOrCreateCache(
+            new CacheConfiguration<>(DEFAULT_CACHE_NAME).setIndexedTypes(Integer.class, Integer.class)
+                .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
     }
 
     /** @throws Exception If failed. */
@@ -75,6 +86,12 @@ public class KillCommandsSQLTest extends GridCommonAbstractTest {
         doTestCancelComputeTask(startCli, srvs, sessId -> execute(killCli, KILL_COMPUTE_QRY + " '" + sessId + "'"));
     }
 
+    /** */
+    @Test
+    public void testCancelTx() {
+        doTestCancelTx(startCli, srvs, xid -> execute(killCli, KILL_TX_QRY + " '" + xid + "'"));
+    }
+
     /** @throws Exception If failed. */
     @Test
     public void testCancelService() throws Exception {
@@ -94,6 +111,12 @@ public class KillCommandsSQLTest extends GridCommonAbstractTest {
         execute(killCli, KILL_SVC_QRY + " 'unknown'");
     }
 
+    /** */
+    @Test
+    public void testCancelUnknownTx() {
+        execute(killCli, KILL_TX_QRY + " 'unknown'");
+    }
+
     /**
      * Execute query on given node.
      *
diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsTests.java b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsTests.java
index 7a0af7f..c580b05 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsTests.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/util/KillCommandsTests.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.function.Consumer;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteFuture;
@@ -29,12 +30,15 @@ import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.services.ServiceContext;
 import org.apache.ignite.spi.systemview.view.ServiceView;
 import org.apache.ignite.spi.systemview.view.SystemView;
+import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.internal.processors.service.IgniteServiceProcessor.SVCS_VIEW;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 import static org.apache.ignite.util.KillCommandsSQLTest.execute;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -44,6 +48,9 @@ class KillCommandsTests {
     /** Service name. */
     public static final String SVC_NAME = "my-svc";
 
+    /** Cache name. */
+    public static final String DEFAULT_CACHE_NAME = "default";
+
     /** Operations timeout. */
     public static final int TIMEOUT = 10_000;
 
@@ -104,6 +111,41 @@ class KillCommandsTests {
     }
 
     /**
+     * Test cancel of the transaction.
+     *
+     * @param cli Client node.
+     * @param srvs Server nodes.
+     * @param txCanceler Transaction cancel closure.
+     */
+    public static void doTestCancelTx(IgniteEx cli, List<IgniteEx> srvs, Consumer<String> txCanceler) {
+        IgniteCache<Object, Object> cache = cli.cache(DEFAULT_CACHE_NAME);
+
+        int testKey = 42;
+
+        try (Transaction tx = cli.transactions().txStart()) {
+            cache.put(testKey, 1);
+
+            List<List<?>> txs = execute(cli, "SELECT xid FROM SYS.TRANSACTIONS");
+
+            assertEquals(1, txs.size());
+
+            String xid = (String)txs.get(0).get(0);
+
+            txCanceler.accept(xid);
+
+            assertThrowsWithCause(tx::commit, IgniteException.class);
+
+            for (int i = 0; i < srvs.size(); i++) {
+                txs = execute(srvs.get(i), "SELECT xid FROM SYS.TRANSACTIONS");
+
+                assertEquals(0, txs.size());
+            }
+        }
+
+        assertNull(cache.get(testKey));
+    }
+
+    /**
      * Test cancel of the service.
      *
      * @param startCli Client node to start service.