You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by as...@apache.org on 2021/11/28 12:53:31 UTC

[ignite-3] branch main updated: IGNITE-15085 Transactions support phase 1: 2PL concurrency control

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

ascherbakov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 70726b1  IGNITE-15085 Transactions support phase 1: 2PL concurrency control
70726b1 is described below

commit 70726b16f6dc4797666afa3145c4b382d8c51e3b
Author: Alexey Scherbakov <al...@gmail.com>
AuthorDate: Sun Nov 28 15:53:12 2021 +0300

    IGNITE-15085 Transactions support phase 1: 2PL concurrency control
---
 .../builder/SortedIndexDefinitionBuilder.java      |    1 +
 .../java/org/apache/ignite/table/KeyValueView.java |   18 +-
 .../org/apache/ignite/tx/IgniteTransactions.java   |   43 +-
 .../java/org/apache/ignite/tx/Transaction.java     |   16 +-
 ...Transactions.java => TransactionException.java} |   27 +-
 .../handler/requests/table/ClientTableCommon.java  |   34 +-
 .../table/ClientTupleDeleteAllExactRequest.java    |    7 +-
 .../table/ClientTupleDeleteAllRequest.java         |    3 +-
 .../requests/table/ClientTupleGetAllRequest.java   |    3 +-
 .../table/ClientTupleInsertAllRequest.java         |    3 +-
 .../ClientTupleInsertAllSchemalessRequest.java     |    3 +-
 .../ignite/internal/client/table/ClientTable.java  |    8 +-
 .../ignite/client/fakes/FakeInternalTable.java     |  150 +-
 .../ignite/client/fakes/FakeSchemaRegistry.java    |    2 +-
 .../internal/tostring/IgniteToStringBuilder.java   |   26 +
 .../internal/testframework/IgniteTestUtils.java    |   13 +-
 .../ItMetaStorageServicePersistenceTest.java       |    8 +-
 .../server/raft/MetaStorageListener.java           |    8 +
 .../internal/metastorage/MetaStorageManager.java   |    4 +-
 .../apache/ignite/network/MessagingService.java    |    4 +-
 .../ignite/internal/network/netty/NettyServer.java |   81 +-
 .../dotnet/Apache.Ignite.Tests/Table/TableTests.cs |    4 +-
 modules/platforms/dotnet/DEVNOTES.md               |    3 +
 .../raft/client/service/RaftGroupListener.java     |   11 +
 .../raft/client/service/RaftGroupService.java      |   10 +
 .../service/ItAbstractListenerSnapshotTest.java    |   25 +-
 .../apache/ignite/raft/jraft/core/ItNodeTest.java  |   60 +-
 .../apache/ignite/raft/server/CounterListener.java |    8 +
 .../raft/server/ItJraftCounterServerTest.java      |   25 +-
 .../java/org/apache/ignite/internal/raft/Loza.java |   57 +-
 .../ignite/internal/raft/server/RaftServer.java    |   10 +
 .../internal/raft/server/impl/JraftServerImpl.java |   20 +-
 .../apache/ignite/raft/jraft/core/NodeImpl.java    |   16 +-
 .../raft/jraft/disruptor/StripedDisruptor.java     |   10 +-
 .../apache/ignite/raft/jraft/error/RaftError.java  |    2 +-
 .../ignite/raft/jraft/option/NodeOptions.java      |    1 +
 .../ignite/raft/jraft/option/ReadOnlyOption.java   |    1 -
 .../ignite/raft/jraft/rpc/RpcResponseClosure.java  |    3 -
 .../jraft/rpc/impl/ActionRequestProcessor.java     |  171 ++-
 .../raft/jraft/rpc/impl/IgniteRpcServer.java       |   48 +-
 .../raft/jraft/rpc/impl/RaftGroupServiceImpl.java  |   48 +-
 .../rpc/impl/cli/AddLearnersRequestProcessor.java  |    2 -
 .../rpc/impl/cli/BaseCliRequestProcessor.java      |    2 -
 .../jraft/rpc/impl/core/NodeRequestProcessor.java  |    1 -
 .../rpc/impl/core/ReadIndexRequestProcessor.java   |    3 -
 .../rpc/impl/core/TimeoutNowRequestProcessor.java  |    3 -
 .../apache/ignite/raft/jraft/storage/Storage.java  |    3 -
 .../raft/jraft/storage/impl/LogManagerImpl.java    |    3 +
 .../apache/ignite/raft/jraft/util/ArrayDeque.java  |    3 -
 .../org/apache/ignite/raft/jraft/util/Bits.java    |    3 -
 .../apache/ignite/raft/jraft/util/Copiable.java    |    3 -
 .../DefaultFixedThreadsExecutorGroupFactory.java   |    2 +-
 .../internal/raft/server/impl/RaftServerImpl.java  |    9 +-
 .../apache/ignite/raft/jraft/core/TestCluster.java |    4 +-
 .../ignite/raft/jraft/util/EndpointTest.java       |    1 -
 .../concurrent/SingleThreadExecutorBenchmark.java  |   19 +-
 .../ItDistributedConfigurationPropertiesTest.java  |    4 +-
 .../ItDistributedConfigurationStorageTest.java     |   72 +-
 .../internal/runner/app/ItNoThreadsLeftTest.java   |   16 +-
 .../org/apache/ignite/internal/app/IgniteImpl.java |   22 +-
 modules/schema/pom.xml                             |    3 +-
 .../ignite/internal/schema/ByteBufferRow.java      |   11 +-
 .../schema/registry/SchemaRegistryImpl.java        |    6 +-
 .../SchemaConfigurationConverterTest.java          |   93 +-
 .../basic/ConcurrentHashMapPartitionStorage.java   |   40 +-
 modules/table/pom.xml                              |    5 +
 .../ignite/distributed/ItDistributedTableTest.java |  606 --------
 .../distributed/ItInternalTableScanTest.java       |   38 +-
 .../ignite/distributed/ItTablePersistenceTest.java |   86 +-
 .../distributed/ItTxDistributedTestSingleNode.java |  437 ++++++
 .../ItTxDistributedTestSingleNodeCollocated.java   |   56 +
 ...ItTxDistributedTestThreeNodesThreeReplicas.java |   55 +
 ...butedTestThreeNodesThreeReplicasCollocated.java |   50 +
 .../ignite/internal/table/AbstractTableView.java   |   28 +-
 .../ignite/internal/table/InternalTable.java       |   49 +-
 .../internal/table/KeyValueBinaryViewImpl.java     |  176 +--
 .../ignite/internal/table/KeyValueViewImpl.java    |   21 +-
 .../internal/table/RecordBinaryViewImpl.java       |  229 ++-
 .../apache/ignite/internal/table/TableImpl.java    |   41 +-
 .../org/apache/ignite/internal/table/TableRow.java |    4 +-
 .../internal/table/distributed/TableManager.java   |   83 +-
 .../table/distributed/TableTxManagerImpl.java      |   56 +
 .../table/distributed/command/CommandUtils.java    |   45 +-
 .../distributed/command/DeleteAllCommand.java      |   42 +-
 .../table/distributed/command/DeleteCommand.java   |   39 +-
 .../distributed/command/DeleteExactAllCommand.java |   42 +-
 .../distributed/command/DeleteExactCommand.java    |   39 +-
 .../table/distributed/command/FinishTxCommand.java |   59 +
 .../table/distributed/command/GetAllCommand.java   |   42 +-
 .../distributed/command/GetAndDeleteCommand.java   |   38 +-
 .../distributed/command/GetAndReplaceCommand.java  |   38 +-
 .../distributed/command/GetAndUpsertCommand.java   |   41 +-
 .../table/distributed/command/GetCommand.java      |   38 +-
 .../distributed/command/InsertAllCommand.java      |   42 +-
 .../table/distributed/command/InsertCommand.java   |   38 +-
 ...teExactAllCommand.java => MultiKeyCommand.java} |   45 +-
 .../table/distributed/command/ReplaceCommand.java  |   47 +-
 .../distributed/command/ReplaceIfExistCommand.java |   37 +-
 .../{DeleteCommand.java => SingleKeyCommand.java}  |   32 +-
 .../distributed/command/TransactionalCommand.java} |   25 +-
 .../distributed/command/UpsertAllCommand.java      |   42 +-
 .../table/distributed/command/UpsertCommand.java   |   38 +-
 .../command/response/MultiRowsResponse.java        |    2 +-
 .../command/response/SingleRowResponse.java        |    2 +-
 .../table/distributed/raft/PartitionListener.java  |  371 +++--
 .../distributed/storage/InternalTableImpl.java     |  325 +++--
 .../distributed/storage/VersionedRowStore.java     |  754 ++++++++++
 .../org/apache/ignite/internal/table/Example.java  |    9 +-
 .../internal/table/InteropOperationsTest.java      |   41 +-
 .../table/KeyValueBinaryViewOperationsTest.java    |  278 ++--
 .../KeyValueViewOperationsSimpleSchemaTest.java    |  173 +--
 .../internal/table/KeyValueViewOperationsTest.java |  213 +--
 .../table/RecordBinaryViewOperationsTest.java      |  205 +--
 .../internal/table/RecordViewOperationsTest.java   |  155 +-
 .../internal/table/SchemaValidationTest.java       |   28 +-
 .../ignite/internal/table/TableManagerTest.java    |   16 +-
 .../ignite/internal/table/TxAbstractTest.java      | 1496 ++++++++++++++++++++
 .../apache/ignite/internal/table/TxLocalTest.java  |   91 ++
 .../org/apache/ignite/internal/table/TxTest.java   |  197 ---
 .../raft/PartitionCommandListenerTest.java         |   52 +-
 .../table/impl/DummyInternalTableImpl.java         |  357 ++---
 .../table/impl/DummySchemaManagerImpl.java         |    4 +-
 modules/transactions/README.md                     |  174 ++-
 modules/transactions/pom.xml                       |  152 +-
 .../ignite/internal/tx/InternalTransaction.java    |   74 +
 .../apache/ignite/internal/tx/LockException.java   |    2 +-
 .../org/apache/ignite/internal/tx/LockManager.java |   17 +-
 .../org/apache/ignite/internal/tx/Timestamp.java   |  140 +-
 .../org/apache/ignite/internal/tx/TxManager.java   |  163 +++
 .../org/apache/ignite/internal/tx/TxState.java}    |   15 +-
 .../java/org/apache/ignite/internal/tx/Waiter.java |   13 +-
 .../ignite/internal/tx/impl/HeapLockManager.java   |  242 +++-
 .../internal/tx/impl/IgniteTransactionsImpl.java   |  100 ++
 .../ignite/internal/tx/impl/TransactionImpl.java   |  194 +++
 .../ignite/internal/tx/impl/TxManagerImpl.java     |  419 ++++++
 .../internal/tx/message/TxFinishRequest.java}      |   38 +-
 .../internal/tx/message/TxFinishResponse.java}     |   25 +-
 .../{Waiter.java => message/TxMessageGroup.java}   |   19 +-
 .../internal/tx/AbstractLockManagerTest.java       |  215 ++-
 .../apache/ignite/internal/tx/TxManagerTest.java   |  141 ++
 parent/pom.xml                                     |    6 +
 141 files changed, 7618 insertions(+), 3452 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/schema/definition/builder/SortedIndexDefinitionBuilder.java b/modules/api/src/main/java/org/apache/ignite/schema/definition/builder/SortedIndexDefinitionBuilder.java
index 7ec4f15..ed425a5 100644
--- a/modules/api/src/main/java/org/apache/ignite/schema/definition/builder/SortedIndexDefinitionBuilder.java
+++ b/modules/api/src/main/java/org/apache/ignite/schema/definition/builder/SortedIndexDefinitionBuilder.java
@@ -39,6 +39,7 @@ public interface SortedIndexDefinitionBuilder extends SchemaObjectBuilder {
     /**
      * Unique index flag.
      *
+     * @param b {@code True} if a unique index.
      * @return {@code This} for chaining.
      */
     SortedIndexDefinitionBuilder unique(boolean b);
diff --git a/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
index 8647f35..1bc79f7 100644
--- a/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
@@ -148,7 +148,7 @@ public interface KeyValueView<K, V> {
      * @param val Value to be associated with the specified key.
      * @return Future representing pending completion of the operation.
      */
-    @NotNull CompletableFuture<Boolean> putIfAbsentAsync(@NotNull K key, V val);
+    @NotNull CompletableFuture<Boolean> putIfAbsentAsync(@NotNull K key, @NotNull V val);
 
     /**
      * Removes value associated with given key from the table.
@@ -231,7 +231,7 @@ public interface KeyValueView<K, V> {
      * @param val Value to be associated with the specified key.
      * @return {@code True} if an old value was replaced, {@code false} otherwise.
      */
-    boolean replace(@NotNull K key, V val);
+    boolean replace(@NotNull K key, @NotNull V val);
 
     /**
      * Replaces the expected value for a key. This is equivalent to
@@ -244,21 +244,21 @@ public interface KeyValueView<K, V> {
      * }</code></pre>
      * except that the action is performed atomically.
      *
-     * @param key    A key with which the specified value is associated. The key cannot be {@code null}.
+     * @param key    A key with which the specified value is associated. The key and values cannot be {@code null}.
      * @param oldVal Expected value associated with the specified key.
      * @param newVal Value to be associated with the specified key.
      * @return {@code True} if an old value was replaced, {@code false} otherwise.
      */
-    boolean replace(@NotNull K key, V oldVal, V newVal);
+    boolean replace(@NotNull K key, @NotNull V oldVal, @NotNull V newVal);
 
     /**
      * Asynchronously replaces the value for a key only if exists. See {@link #replace(Object, Object)}.
      *
-     * @param key A key with which the specified value is associated. The key cannot be {@code null}.
+     * @param key A key with which the specified value is associated. The key and value cannot be {@code null}.
      * @param val Value to be associated with the specified key.
      * @return Future representing pending completion of the operation.
      */
-    @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, V val);
+    @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, @NotNull V val);
 
     /**
      * Asynchronously replaces the expected value for a key. See {@link #replace(Object, Object, Object)}
@@ -268,7 +268,7 @@ public interface KeyValueView<K, V> {
      * @param newVal Value to be associated with the specified key.
      * @return Future representing pending completion of the operation.
      */
-    @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, V oldVal, V newVal);
+    @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, @NotNull V oldVal, @NotNull V newVal);
 
     /**
      * Replaces the value for a given key only if exists. This is equivalent to
@@ -287,7 +287,7 @@ public interface KeyValueView<K, V> {
      * @param val Value to be associated with the specified key.
      * @return Replaced value, or {@code null} if not existed.
      */
-    V getAndReplace(@NotNull K key, V val);
+    V getAndReplace(@NotNull K key, @NotNull V val);
 
     /**
      * Asynchronously replaces the value for a given key only if exists. See {@link #getAndReplace(Object, Object)}
@@ -296,7 +296,7 @@ public interface KeyValueView<K, V> {
      * @param val Value to be associated with the specified key.
      * @return Future representing pending completion of the operation.
      */
-    @NotNull CompletableFuture<V> getAndReplaceAsync(@NotNull K key, V val);
+    @NotNull CompletableFuture<V> getAndReplaceAsync(@NotNull K key, @NotNull V val);
 
     /**
      * Executes invoke processor code against the value associated with the provided key.
diff --git a/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java b/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
index 13ccdd2..3fcbc5b 100644
--- a/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
+++ b/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
@@ -19,24 +19,59 @@ package org.apache.ignite.tx;
 
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Consumer;
+import java.util.function.Function;
 
 /**
  * Ignite Transactions facade.
  */
 public interface IgniteTransactions {
     /**
+     * Returns a facade with new default timeout.
+     *
+     * @param timeout The timeout in milliseconds.
+     *
+     * @return A facade using a new timeout.
+     */
+    IgniteTransactions withTimeout(long timeout);
+
+    /**
      * Begins a transaction.
      *
-     * @return The future.
+     * @return The started transaction.
+     */
+    Transaction begin();
+
+    /**
+     * Begins an async transaction.
+     *
+     * @return The future holding the started transaction.
      */
     CompletableFuture<Transaction> beginAsync();
 
     /**
-     * Synchronously executes a closure within a transaction.
+     * Executes a closure within a transaction.
      *
-     * <p>If the closure is executed normally (no exceptions), the transaction is automatically committed.
+     * <p>If the closure is executed normally (no exceptions) the transaction is automatically committed.
      *
      * @param clo The closure.
+     *
+     * @throws TransactionException If a transaction can't be finished successfully.
+     */
+    void runInTransaction(Consumer<Transaction> clo) throws TransactionException;
+
+    /**
+     * Executes a closure within a transaction and returns a result.
+     *
+     * <p>If the closure is executed normally (no exceptions) the transaction is automatically committed.
+     *
+     * <p>This method will automatically enlist all tables into the transaction, but the execution of
+     * the transaction shouldn't leave starting thread or an exception will be thrown.
+     *
+     * @param clo The closure.
+     * @param <T> Closure result type.
+     * @return The result.
+     *
+     * @throws TransactionException If a transaction can't be finished successfully.
      */
-    void runInTransaction(Consumer<Transaction> clo);
+    <T> T runInTransaction(Function<Transaction, T> clo) throws TransactionException;
 }
diff --git a/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java b/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
index 1825409..29c2b3a 100644
--- a/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
+++ b/modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
@@ -24,24 +24,28 @@ import java.util.concurrent.CompletableFuture;
  */
 public interface Transaction {
     /**
-     * Synchronously commits a transaction. Does nothing if it's already finished by commiting or rolling back.
+     * Synchronously commits a transaction.
+     *
+     * @throws TransactionException If a transaction can't be commited.
      */
-    void commit();
+    void commit() throws TransactionException;
 
     /**
-     * Asynchronously commits a transaction. Does nothing if it's already finished by commiting or rolling back.
+     * Asynchronously commits a transaction.
      *
      * @return The future.
      */
     CompletableFuture<Void> commitAsync();
 
     /**
-     * Synchronously rolls back a transaction. Does nothing if it's already finished by commiting or rolling back.
+     * Synchronously rolls back a transaction.
+     *
+     * @throws TransactionException If a transaction can't be rolled back.
      */
-    void rollback();
+    void rollback() throws TransactionException;
 
     /**
-     * Asynchronously rolls back a transaction. Does nothing if it's already finished by commiting or rolling back.
+     * Asynchronously rolls back a transaction.
      *
      * @return The future.
      */
diff --git a/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java b/modules/api/src/main/java/org/apache/ignite/tx/TransactionException.java
similarity index 62%
copy from modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
copy to modules/api/src/main/java/org/apache/ignite/tx/TransactionException.java
index 13ccdd2..7b60d6d 100644
--- a/modules/api/src/main/java/org/apache/ignite/tx/IgniteTransactions.java
+++ b/modules/api/src/main/java/org/apache/ignite/tx/TransactionException.java
@@ -17,26 +17,25 @@
 
 package org.apache.ignite.tx;
 
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteException;
 
-/**
- * Ignite Transactions facade.
- */
-public interface IgniteTransactions {
+/** This exception is thrown if a transaction can't be finished by some reasons. */
+public class TransactionException extends IgniteException {
     /**
-     * Begins a transaction.
+     * Creates a new transaction exception with a message.
      *
-     * @return The future.
+     * @param message The message.
      */
-    CompletableFuture<Transaction> beginAsync();
+    public TransactionException(String message) {
+        super(message);
+    }
 
     /**
-     * Synchronously executes a closure within a transaction.
-     *
-     * <p>If the closure is executed normally (no exceptions), the transaction is automatically committed.
+     * Creates a new transaction exception with a cause.
      *
-     * @param clo The closure.
+     * @param cause The cause.
      */
-    void runInTransaction(Consumer<Transaction> clo);
+    public TransactionException(Throwable cause) {
+        super(cause);
+    }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
index 3a3be62..e8ee057 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.NativeTypeSpec;
 import org.apache.ignite.internal.schema.SchemaAware;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.table.IgniteTablesInternal;
 import org.apache.ignite.internal.table.TableImpl;
 import org.apache.ignite.lang.IgniteException;
@@ -193,10 +194,11 @@ class ClientTableCommon {
      *
      * @param packer Packer.
      * @param tuples Tuples.
+     * @param schemaRegistry The registry.
      * @throws IgniteException on failed serialization.
      */
-    public static void writeTuples(ClientMessagePacker packer, Collection<Tuple> tuples) {
-        writeTuples(packer, tuples, TuplePart.KEY_AND_VAL);
+    public static void writeTuples(ClientMessagePacker packer, Collection<Tuple> tuples, SchemaRegistry schemaRegistry) {
+        writeTuples(packer, tuples, TuplePart.KEY_AND_VAL, schemaRegistry);
     }
     
     /**
@@ -205,27 +207,31 @@ class ClientTableCommon {
      * @param packer Packer.
      * @param tuples Tuples.
      * @param part   Which part of tuple to write.
+     * @param schemaRegistry The registry.
      * @throws IgniteException on failed serialization.
      */
-    public static void writeTuples(ClientMessagePacker packer, Collection<Tuple> tuples, TuplePart part) {
+    public static void writeTuples(
+            ClientMessagePacker packer,
+            Collection<Tuple> tuples,
+            TuplePart part,
+            SchemaRegistry schemaRegistry
+    ) {
         if (tuples == null || tuples.isEmpty()) {
             packer.packNil();
-            
+        
             return;
         }
-        
-        SchemaDescriptor schema = null;
-        
-        for (Tuple tuple : tuples) {
-            if (schema == null) {
-                schema = ((SchemaAware) tuple).schema();
     
-                packer.packInt(schema.version());
-                packer.packInt(tuples.size());
-            } else {
+        SchemaDescriptor schema = schemaRegistry.schema();
+    
+        packer.packInt(schema.version());
+        packer.packInt(tuples.size());
+    
+        for (Tuple tuple : tuples) {
+            if (tuple != null) {
                 assert schema.version() == ((SchemaAware) tuple).schema().version();
             }
-            
+        
             writeTuple(packer, tuple, schema, true, part);
         }
     }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllExactRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllExactRequest.java
index 20deaa2..a1104d3 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllExactRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllExactRequest.java
@@ -33,8 +33,8 @@ public class ClientTupleDeleteAllExactRequest {
     /**
      * Processes the request.
      *
-     * @param in     Unpacker.
-     * @param out    Packer.
+     * @param in Unpacker.
+     * @param out Packer.
      * @param tables Ignite tables.
      * @return Future.
      */
@@ -46,6 +46,7 @@ public class ClientTupleDeleteAllExactRequest {
         var table = readTable(in, tables);
         var tuples = readTuples(in, table, false);
 
-        return table.recordView().deleteAllExactAsync(tuples).thenAccept(skippedTuples -> writeTuples(out, skippedTuples));
+        return table.recordView().deleteAllExactAsync(tuples)
+                .thenAccept(skippedTuples -> writeTuples(out, skippedTuples, table.schemaView()));
     }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllRequest.java
index 37b46ff..804b52b 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleDeleteAllRequest.java
@@ -47,6 +47,7 @@ public class ClientTupleDeleteAllRequest {
         var table = readTable(in, tables);
         var tuples = readTuples(in, table, true);
 
-        return table.recordView().deleteAllAsync(tuples).thenAccept(skippedTuples -> writeTuples(out, skippedTuples, TuplePart.KEY));
+        return table.recordView().deleteAllAsync(tuples).thenAccept(skippedTuples ->
+            writeTuples(out, skippedTuples, TuplePart.KEY, table.schemaView()));
     }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleGetAllRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleGetAllRequest.java
index dab0122..4ac6452 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleGetAllRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleGetAllRequest.java
@@ -46,6 +46,7 @@ public class ClientTupleGetAllRequest {
         var table = readTable(in, tables);
         var keyTuples = readTuples(in, table, true);
 
-        return table.recordView().getAllAsync(keyTuples).thenAccept(tuples -> writeTuples(out, tuples));
+        return table.recordView().getAllAsync(keyTuples).thenAccept(tuples ->
+            writeTuples(out, tuples, table.schemaView()));
     }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllRequest.java
index e021eb3..087a10e 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllRequest.java
@@ -46,6 +46,7 @@ public class ClientTupleInsertAllRequest {
         var table = readTable(in, tables);
         var tuples = readTuples(in, table, false);
 
-        return table.recordView().insertAllAsync(tuples).thenAccept(skippedTuples -> writeTuples(out, skippedTuples));
+        return table.recordView().insertAllAsync(tuples).thenAccept(skippedTuples ->
+            writeTuples(out, skippedTuples, table.schemaView()));
     }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllSchemalessRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllSchemalessRequest.java
index b0d733b..ef3456b 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllSchemalessRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTupleInsertAllSchemalessRequest.java
@@ -46,6 +46,7 @@ public class ClientTupleInsertAllSchemalessRequest {
         var table = readTable(in, tables);
         var tuples = readTuplesSchemaless(in);
 
-        return table.recordView().insertAllAsync(tuples).thenAccept(skippedTuples -> writeTuples(out, skippedTuples));
+        return table.recordView().insertAllAsync(tuples).thenAccept(skippedTuples ->
+            writeTuples(out, skippedTuples, table.schemaView()));
     }
 }
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
index 8dcbe2b..ecb0ae8 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
@@ -411,6 +411,10 @@ public class ClientTable implements Table {
     }
     
     static IgniteBiTuple<Tuple, Tuple> readKvTuple(ClientSchema schema, ClientMessageUnpacker in) {
+        if (in.tryUnpackNil()) {
+            return null;
+        }
+        
         var keyColCnt = schema.keyColumnCount();
         var colCnt = schema.columns().length;
         
@@ -444,7 +448,9 @@ public class ClientTable implements Table {
         
         for (int i = 0; i < cnt; i++) {
             var pair = readKvTuple(schema, in);
-            res.put(pair.get1(), pair.get2());
+            if (pair != null) {
+                res.put(pair.get1(), pair.get2());
+            }
         }
         
         return res;
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
index 9c1dbd5..485db8d 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
@@ -28,9 +28,9 @@ import javax.naming.OperationNotSupportedException;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.tx.InternalTransaction;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -40,15 +40,15 @@ import org.jetbrains.annotations.Nullable;
 public class FakeInternalTable implements InternalTable {
     /** Table name. */
     private final String tableName;
-    
+
     /** Table ID. */
     private final IgniteUuid tableId;
-    
+
     /** Table data. */
     private final ConcurrentHashMap<ByteBuffer, BinaryRow> data = new ConcurrentHashMap<>();
-    
+
     /**
-     * Constructor.
+     * The constructor.
      *
      * @param tableName Name.
      * @param tableId   Id.
@@ -57,218 +57,228 @@ public class FakeInternalTable implements InternalTable {
         this.tableName = tableName;
         this.tableId = tableId;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull TableStorage storage() {
         throw new UnsupportedOperationException("Not implemented yet");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public int partitions() {
         return 1;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull IgniteUuid tableId() {
         return tableId;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull String name() {
         return tableName;
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> get(BinaryRow keyRow, @Nullable Transaction tx) {
+    public CompletableFuture<BinaryRow> get(BinaryRow keyRow, @Nullable InternalTransaction tx) {
         return CompletableFuture.completedFuture(data.get(keyRow.keySlice()));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows, @Nullable Transaction tx) {
+    public CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows,
+            @Nullable InternalTransaction tx) {
         var res = new ArrayList<BinaryRow>();
-        
+
         for (var key : keyRows) {
             var val = get(key, null);
-    
+
             if (val != null) {
                 res.add(val.getNow(null));
             }
         }
-        
+
         return CompletableFuture.completedFuture(res);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Void> upsert(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<Void> upsert(BinaryRow row, @Nullable InternalTransaction tx) {
         data.put(row.keySlice(), row);
-        
+
         return CompletableFuture.completedFuture(null);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, @Nullable Transaction tx) {
+    public CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx) {
         for (var row : rows) {
             upsert(row, tx);
         }
-        
+
         return CompletableFuture.completedFuture(null);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row,
+            @Nullable InternalTransaction tx) {
         var res = get(row, tx);
-        
+
         upsert(row, tx);
-        
+
         return CompletableFuture.completedFuture(res.getNow(null));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> insert(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<Boolean> insert(BinaryRow row, @Nullable InternalTransaction tx) {
         var old = get(row, tx).getNow(null);
-        
+
         if (old == null) {
             upsert(row, tx);
-            
+
             return CompletableFuture.completedFuture(true);
         }
-        
+
         return CompletableFuture.completedFuture(false);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, @Nullable Transaction tx) {
+    public CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx) {
         var skipped = new ArrayList<BinaryRow>();
-        
+
         for (var row : rows) {
             if (!insert(row, tx).getNow(null)) {
                 skipped.add(row);
             }
         }
-        
+
         return CompletableFuture.completedFuture(skipped);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> replace(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<Boolean> replace(BinaryRow row, @Nullable InternalTransaction tx) {
         var old = get(row, tx).getNow(null);
-    
+
         if (old == null) {
             return CompletableFuture.completedFuture(false);
         }
-        
+
         return upsert(row, tx).thenApply(f -> true);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow, @Nullable Transaction tx) {
+    public CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow, @Nullable InternalTransaction tx) {
         var old = get(oldRow, tx).getNow(null);
-    
+
         if (old == null || !old.valueSlice().equals(oldRow.valueSlice())) {
             return CompletableFuture.completedFuture(false);
         }
-        
+
         return upsert(newRow, tx).thenApply(f -> true);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndReplace(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<BinaryRow> getAndReplace(BinaryRow row,
+            @Nullable InternalTransaction tx) {
         var old = get(row, tx);
-        
+
         return replace(row, tx).thenCompose(f -> old);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> delete(BinaryRow keyRow, @Nullable Transaction tx) {
+    public CompletableFuture<Boolean> delete(BinaryRow keyRow, @Nullable InternalTransaction tx) {
         var old = get(keyRow, tx).getNow(null);
-    
+
         if (old != null) {
             data.remove(keyRow.keySlice());
         }
-        
+
         return CompletableFuture.completedFuture(old != null);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, @Nullable Transaction tx) {
+    public CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, @Nullable InternalTransaction tx) {
         var old = get(oldRow, tx).getNow(null);
-        
+
         if (old != null && old.valueSlice().equals(oldRow.valueSlice())) {
             data.remove(oldRow.keySlice());
             return CompletableFuture.completedFuture(true);
         }
-        
+
         return CompletableFuture.completedFuture(false);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndDelete(BinaryRow row, @Nullable Transaction tx) {
+    public CompletableFuture<BinaryRow> getAndDelete(BinaryRow row,
+            @Nullable InternalTransaction tx) {
         var old = get(row, tx).getNow(null);
-    
+
         if (old != null) {
             data.remove(row.keySlice());
         }
-        
+
         return CompletableFuture.completedFuture(old);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, @Nullable Transaction tx) {
+    public CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx) {
         var skipped = new ArrayList<BinaryRow>();
-        
+
         for (var row : rows) {
             if (!delete(row, tx).getNow(false)) {
                 skipped.add(row);
             }
         }
-        
+
         return CompletableFuture.completedFuture(skipped);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> deleteAllExact(Collection<BinaryRow> rows, @Nullable Transaction tx) {
+    public CompletableFuture<Collection<BinaryRow>> deleteAllExact(Collection<BinaryRow> rows, @Nullable InternalTransaction tx) {
         var skipped = new ArrayList<BinaryRow>();
-        
+
         for (var row : rows) {
             if (!deleteExact(row, tx).getNow(false)) {
                 skipped.add(row);
             }
         }
-        
+
         return CompletableFuture.completedFuture(skipped);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public @NotNull Publisher<BinaryRow> scan(int p, @Nullable Transaction tx) {
+    public @NotNull Publisher<BinaryRow> scan(int p, @Nullable InternalTransaction tx) {
         throw new IgniteInternalException(new OperationNotSupportedException());
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull List<String> assignments() {
         throw new IgniteInternalException(new OperationNotSupportedException());
     }
-    
+
+    /** {@inheritDoc} */
+    @Override
+    public int partition(BinaryRow keyRow) {
+        return 0;
+    }
+
     /** {@inheritDoc} */
     @Override
     public void close() throws Exception {
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeSchemaRegistry.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeSchemaRegistry.java
index 6f2eb3c..1cc3ce7 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeSchemaRegistry.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeSchemaRegistry.java
@@ -106,7 +106,7 @@ public class FakeSchemaRegistry implements SchemaRegistry {
     /** {@inheritDoc} */
     @Override
     public Row resolve(BinaryRow row) {
-        return new Row(schema(row.schemaVersion()), row);
+        return row == null ? null : new Row(schema(row.schemaVersion()), row);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/tostring/IgniteToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/tostring/IgniteToStringBuilder.java
index 79d233a..47a6e47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/tostring/IgniteToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/tostring/IgniteToStringBuilder.java
@@ -836,6 +836,32 @@ public class IgniteToStringBuilder {
     }
 
     /**
+     * Produces auto-generated output of string presentation for given object.
+     *
+     * @param obj Object to get a string presentation for.
+     * @return String presentation of the given object.
+     */
+    public static String toString(@Nullable Object obj) {
+        if (obj == null) {
+            return "null";
+        }
+
+        Class cls = obj.getClass();
+
+        StringBuilderLimitedLength sb = threadLocSB.get();
+
+        boolean newStr = sb.length() == 0;
+
+        try {
+            return toStringImpl(cls, sb, obj, EMPTY_ARRAY, EMPTY_ARRAY, null, 0);
+        } finally {
+            if (newStr) {
+                sb.reset();
+            }
+        }
+    }
+
+    /**
      * Produces auto-generated output of string presentation for given object and its declaration class.
      *
      * @param <T>    Type of the object.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java
index ad48b1c..cfb852b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/testframework/IgniteTestUtils.java
@@ -336,14 +336,19 @@ public final class IgniteTestUtils {
 
         return sb.toString();
     }
-
+    
     /**
      * Creates a unique Ignite node name for the given test.
+     *
+     * @param testInfo Test info.
+     * @param idx Node index.
+     *
+     * @return Node name.
      */
     public static String testNodeName(TestInfo testInfo, int idx) {
         return LoggerMessageHelper.format("{}_{}_{}",
-            testInfo.getTestClass().map(Class::getSimpleName).orElseGet(() -> "null"),
-            testInfo.getTestMethod().map(Method::getName).orElseGet(() -> "null"),
-            idx);
+                testInfo.getTestClass().map(Class::getSimpleName).orElseGet(() -> "null"),
+                testInfo.getTestMethod().map(Method::getName).orElseGet(() -> "null"),
+                idx);
     }
 }
diff --git a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServicePersistenceTest.java b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServicePersistenceTest.java
index 05863d6..5b2781c 100644
--- a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServicePersistenceTest.java
+++ b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServicePersistenceTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.raft.client.service.ItAbstractListenerSnapshotTest;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.apache.ignite.raft.client.service.RaftGroupService;
@@ -53,9 +54,7 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
     private KeyValueStorage storage;
 
-    /**
-     * After each.
-     */
+    /** After each. */
     @AfterEach
     void tearDown() throws Exception {
         if (storage != null) {
@@ -73,7 +72,6 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
         // Check that data has been written successfully
         check(metaStorage, new EntryImpl(FIRST_KEY, FIRST_VALUE, 1, 1));
-        ;
     }
 
     /** {@inheritDoc} */
@@ -130,7 +128,7 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
     /** {@inheritDoc} */
     @Override
-    public RaftGroupListener createListener(Path listenerPersistencePath) {
+    public RaftGroupListener createListener(ClusterService service, Path listenerPersistencePath) {
         storage = new RocksDbKeyValueStorage(listenerPersistencePath);
 
         storage.start();
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
index a4c1c22..24d5804 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.metastorage.common.ConditionType;
@@ -62,10 +63,12 @@ import org.apache.ignite.internal.metastorage.server.WatchEvent;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.jetbrains.annotations.Nullable;
 import org.jetbrains.annotations.TestOnly;
 
 /**
@@ -377,6 +380,11 @@ public class MetaStorageListener implements RaftGroupListener {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public @Nullable CompletableFuture<Void> onBeforeApply(Command command) {
+        return null;
+    }
+
     /**
      * Returns {@link KeyValueStorage} that is backing this listener.
      */
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index 2985177..56c3dd4 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -969,9 +969,7 @@ public class MetaStorageManager implements IgniteComponent {
          *
          * @param innerCursorFut Inner cursor future.
          */
-        CursorWrapper(
-                CompletableFuture<Cursor<T>> innerCursorFut
-        ) {
+        CursorWrapper(CompletableFuture<Cursor<T>> innerCursorFut) {
             this.innerCursorFut = innerCursorFut;
             this.innerIterFut = innerCursorFut.thenApply(Iterable::iterator);
         }
diff --git a/modules/network-api/src/main/java/org/apache/ignite/network/MessagingService.java b/modules/network-api/src/main/java/org/apache/ignite/network/MessagingService.java
index 0b3621c..d27b3fc 100644
--- a/modules/network-api/src/main/java/org/apache/ignite/network/MessagingService.java
+++ b/modules/network-api/src/main/java/org/apache/ignite/network/MessagingService.java
@@ -79,7 +79,7 @@ public interface MessagingService {
      * @param timeout   Waiting for response timeout in milliseconds.
      * @return A future holding the response or error if the expected response was not received.
      */
-    CompletableFuture<NetworkMessage> invoke(ClusterNode recipient, NetworkMessage msg, long timeout);
+    <T extends NetworkMessage> CompletableFuture<T> invoke(ClusterNode recipient, NetworkMessage msg, long timeout);
 
     /**
      * Sends a message asynchronously with same guarantees as {@link #send(ClusterNode, NetworkMessage)} and returns a future that will be
@@ -90,7 +90,7 @@ public interface MessagingService {
      * @param timeout Waiting for response timeout in milliseconds.
      * @return A future holding the response or error if the expected response was not received.
      */
-    CompletableFuture<NetworkMessage> invoke(NetworkAddress addr, NetworkMessage msg, long timeout);
+    <T extends NetworkMessage> CompletableFuture<T> invoke(NetworkAddress addr, NetworkMessage msg, long timeout);
 
     /**
      * Registers a listener for a group of network message events.
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyServer.java b/modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyServer.java
index 985a483..d4e48d1 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyServer.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/netty/NettyServer.java
@@ -46,39 +46,39 @@ import org.jetbrains.annotations.TestOnly;
 public class NettyServer {
     /** A lock for start and stop operations. */
     private final Object startStopLock = new Object();
-    
+
     /** Bootstrap factory. */
     private final NettyBootstrapFactory bootstrapFactory;
-    
+
     /** Server socket configuration. */
     private final NetworkView configuration;
-    
+
     /** Serialization registry. */
     private final MessageSerializationRegistry serializationRegistry;
-    
+
     /** Incoming message listener. */
     private final BiConsumer<SocketAddress, NetworkMessage> messageListener;
-    
+
     /** Handshake manager. */
     private final Supplier<HandshakeManager> handshakeManager;
-    
+
     /** Server start future. */
     private CompletableFuture<Void> serverStartFuture;
-    
+
     /** Server socket channel. */
     @Nullable
     private volatile ServerChannel channel;
-    
+
     /** Server close future. */
     @Nullable
     private CompletableFuture<Void> serverCloseFuture;
-    
+
     /** New connections listener. */
     private final Consumer<NettySender> newConnectionListener;
-    
+
     /** Flag indicating if {@link #stop()} has been called. */
     private boolean stopped;
-    
+
     /**
      * Constructor.
      *
@@ -105,7 +105,7 @@ public class NettyServer {
         this.serializationRegistry = serializationRegistry;
         this.bootstrapFactory = bootstrapFactory;
     }
-    
+
     /**
      * Starts the server.
      *
@@ -116,20 +116,20 @@ public class NettyServer {
             if (stopped) {
                 throw new IgniteInternalException("Attempted to start an already stopped server");
             }
-    
+
             if (serverStartFuture != null) {
                 throw new IgniteInternalException("Attempted to start an already started server");
             }
-    
+
             ServerBootstrap bootstrap = bootstrapFactory.createServerBootstrap();
-            
+
             bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
                         /** {@inheritDoc} */
                         @Override
                         public void initChannel(SocketChannel ch) {
                             // Get handshake manager for the new channel.
                             HandshakeManager manager = handshakeManager.get();
-                            
+
                             ch.pipeline().addLast(
                                     /*
                                      * Decoder that uses the MessageReader
@@ -149,30 +149,30 @@ public class NettyServer {
                                     new OutboundEncoder(serializationRegistry),
                                     new IoExceptionSuppressingHandler()
                             );
-                            
+
                             manager.handshakeFuture().thenAccept(newConnectionListener);
                         }
                     });
-            
+
             int port = configuration.port();
             int portRange = configuration.portRange();
-            
+
             var bindFuture = new CompletableFuture<Channel>();
-            
-            tryBind(bootstrap, port, port + portRange, bindFuture);
-            
+
+            tryBind(bootstrap, port, port + portRange, port, bindFuture);
+
             serverStartFuture = bindFuture
                     .handle((channel, err) -> {
                         synchronized (startStopLock) {
                             if (channel != null) {
                                 serverCloseFuture = NettyUtils.toCompletableFuture(channel.closeFuture());
                             }
-                            
+
                             this.channel = (ServerChannel) channel;
-    
+
                             if (err != null || stopped) {
                                 Throwable stopErr = err != null ? err : new CancellationException("Server was stopped");
-                                
+
                                 return CompletableFuture.<Void>failedFuture(stopErr);
                             } else {
                                 return CompletableFuture.<Void>completedFuture(null);
@@ -180,35 +180,36 @@ public class NettyServer {
                         }
                     })
                     .thenCompose(Function.identity());
-            
+
             return serverStartFuture;
         }
     }
-    
+
     /**
      * Try bind this server to a port.
      *
      * @param bootstrap Bootstrap.
      * @param port      Target port.
      * @param endPort   Last port that server can be bound to.
+     * @param startPort Start port.
      * @param fut       Future.
      */
-    private void tryBind(ServerBootstrap bootstrap, int port, int endPort, CompletableFuture<Channel> fut) {
+    private void tryBind(ServerBootstrap bootstrap, int port, int endPort, int startPort, CompletableFuture<Channel> fut) {
         if (port > endPort) {
-            fut.completeExceptionally(new IllegalStateException("No available port in range"));
+            fut.completeExceptionally(new IllegalStateException("No available port in range [" + startPort + "-" + endPort + ']'));
         }
-        
+
         bootstrap.bind(port).addListener((ChannelFuture future) -> {
             if (future.isSuccess()) {
                 fut.complete(future.channel());
             } else if (future.isCancelled()) {
                 fut.cancel(true);
             } else {
-                tryBind(bootstrap, port + 1, endPort, fut);
+                tryBind(bootstrap, port + 1, endPort, startPort, fut);
             }
         });
     }
-    
+
     /**
      * Returns gets the local address of the server.
      *
@@ -217,7 +218,7 @@ public class NettyServer {
     public SocketAddress address() {
         return channel.localAddress();
     }
-    
+
     /**
      * Stops the server.
      *
@@ -228,25 +229,25 @@ public class NettyServer {
             if (stopped) {
                 return CompletableFuture.completedFuture(null);
             }
-            
+
             stopped = true;
-    
+
             if (serverStartFuture == null) {
                 return CompletableFuture.completedFuture(null);
             }
-    
+
             var serverCloseFuture0 = serverCloseFuture;
-            
+
             return serverStartFuture.handle((unused, throwable) -> {
                 if (channel != null) {
                     channel.close();
                 }
-                
+
                 return serverCloseFuture0 == null ? CompletableFuture.<Void>completedFuture(null) : serverCloseFuture0;
             }).thenCompose(Function.identity());
         }
     }
-    
+
     /**
      * Returns {@code true} if the server is running, {@code false} otherwise.
      *
@@ -255,7 +256,7 @@ public class NettyServer {
     @TestOnly
     public boolean isRunning() {
         var channel0 = channel;
-        
+
         return channel0 != null && channel0.isOpen();
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Table/TableTests.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/TableTests.cs
index c04a3dd..bd5e6ea 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/Table/TableTests.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Table/TableTests.cs
@@ -351,6 +351,7 @@ namespace Apache.Ignite.Tests.Table
         }
 
         [Test]
+        [Ignore("IGNITE-15939 TODO asch read null tuples")]
         public async Task TestGetAllReturnsRecordsForExistingKeys()
         {
             var records = Enumerable
@@ -362,7 +363,7 @@ namespace Apache.Ignite.Tests.Table
             var res = await Table.GetAllAsync(Enumerable.Range(9, 4).Select(x => GetTuple(x)));
             var resArr = res.OrderBy(x => x[0]).ToArray();
 
-            Assert.AreEqual(2, res.Count);
+            Assert.AreEqual(4, res.Count);
 
             Assert.AreEqual(9, resArr[0][0]);
             Assert.AreEqual("9", resArr[0][1]);
@@ -372,6 +373,7 @@ namespace Apache.Ignite.Tests.Table
         }
 
         [Test]
+        [Ignore("IGNITE-15939 TODO asch read null tuples")]
         public async Task TestGetAllNonExistentKeysReturnsEmptyList()
         {
             var res = await Table.GetAllAsync(new[] { GetTuple(-100) });
diff --git a/modules/platforms/dotnet/DEVNOTES.md b/modules/platforms/dotnet/DEVNOTES.md
index 1acc99f..f521f42 100644
--- a/modules/platforms/dotnet/DEVNOTES.md
+++ b/modules/platforms/dotnet/DEVNOTES.md
@@ -16,6 +16,9 @@ In this dir: `dotnet test`
 * cd `modules/runner`
 * `mvn exec:java@platform-test-node-runner`
 
+To debug or profile Java side of the tests, run `org.apache.ignite.internal.runner.app.PlatformTestNodeRunner` class in IDEA with a debugger or profiler,
+then run .NET tests with `dotnet test` or `dotnet test --filter TEST_NAME`. When a server node is present, .NET tests will use it instead of starting a new one.
+
 ## .NET Core 3.1 and .NET Standard 2.1
 
 * Library project target `netstandard2.1`
diff --git a/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupListener.java b/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupListener.java
index f44adcb..8035d39 100644
--- a/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupListener.java
+++ b/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupListener.java
@@ -19,9 +19,12 @@ package org.apache.ignite.raft.client.service;
 
 import java.nio.file.Path;
 import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
 import java.util.function.Consumer;
+import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * A listener for replication group events.
@@ -70,4 +73,12 @@ public interface RaftGroupListener {
      * Invoked once after a raft node has been shut down.
      */
     void onShutdown();
+
+    /**
+     * Invoked before submitting a command to a raft group.
+     *
+     * @param command The command.
+     * @return The future or null if no-op.
+     */
+    @Nullable CompletableFuture<Void> onBeforeApply(Command command);
 }
diff --git a/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupService.java b/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupService.java
index f938f90..9fc2674 100644
--- a/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupService.java
+++ b/modules/raft-client/src/main/java/org/apache/ignite/raft/client/service/RaftGroupService.java
@@ -20,11 +20,13 @@ package org.apache.ignite.raft.client.service;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeoutException;
+import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
+import org.jetbrains.annotations.TestOnly;
 
 /**
  * A service providing operations on a replication group.
@@ -225,4 +227,12 @@ public interface RaftGroupService {
      * Shutdown and cleanup resources for this instance.
      */
     void shutdown();
+
+    /**
+     * Returns a cluster service.
+     *
+     * @return Cluster service.
+     */
+    @TestOnly
+    ClusterService clusterService();
 }
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ItAbstractListenerSnapshotTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ItAbstractListenerSnapshotTest.java
index 846b351..397e836 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ItAbstractListenerSnapshotTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/client/service/ItAbstractListenerSnapshotTest.java
@@ -90,7 +90,7 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
     private final List<ClusterService> cluster = new ArrayList<>();
 
     /** Servers. */
-    private final List<JraftServerImpl> servers = new ArrayList<>();
+    protected final List<JraftServerImpl> servers = new ArrayList<>();
 
     /** Clients. */
     private final List<RaftGroupService> clients = new ArrayList<>();
@@ -295,10 +295,11 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
     /**
      * Creates raft group listener.
      *
+     * @param service                 The cluster service.
      * @param listenerPersistencePath Path to storage persistent data.
      * @return Raft group listener.
      */
-    public abstract RaftGroupListener createListener(Path listenerPersistencePath);
+    public abstract RaftGroupListener createListener(ClusterService service, Path listenerPersistencePath);
 
     /**
      * Returns raft group id for tests.
@@ -390,14 +391,14 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
 
         Path listenerPersistencePath = workDir.resolve("db" + idx);
 
+        servers.add(server);
+
         server.startRaftGroup(
                 raftGroupId(),
-                createListener(listenerPersistencePath),
+                createListener(service, listenerPersistencePath),
                 INITIAL_CONF
         );
 
-        servers.add(server);
-
         return server;
     }
 
@@ -417,7 +418,18 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
     }
 
     /**
+     * Returns a client service.
+     *
+     * @return The client service.
+     */
+    protected ClusterService clientService() {
+        return cluster.get(INITIAL_CONF.size());
+    }
+
+    /**
      * Starts a client with a specific address.
+     *
+     * @return The service.
      */
     private RaftGroupService startClient(TestInfo testInfo, String groupId, NetworkAddress addr) throws Exception {
         ClusterService clientNode = clusterService(testInfo, CLIENT_PORT + clients.size(), addr);
@@ -425,6 +437,9 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
         RaftGroupService client = RaftGroupServiceImpl.start(groupId, clientNode, FACTORY, 10_000,
                 List.of(new Peer(addr)), false, 200, executor).get(3, TimeUnit.SECONDS);
 
+        // Transactios by now require a leader to build a mapping.
+        client.refreshLeader().join();
+
         clients.add(client);
 
         return client;
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
index dd13cc9..8240aac 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.ignite.raft.jraft.core;
 
+import com.codahale.metrics.ConsoleReporter;
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.nio.file.Path;
@@ -27,6 +28,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.Vector;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -37,7 +39,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BooleanSupplier;
 import java.util.stream.Stream;
-import com.codahale.metrics.ConsoleReporter;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.lang.IgniteLogger;
@@ -71,6 +72,7 @@ import org.apache.ignite.raft.jraft.error.RaftException;
 import org.apache.ignite.raft.jraft.option.BootstrapOptions;
 import org.apache.ignite.raft.jraft.option.NodeOptions;
 import org.apache.ignite.raft.jraft.option.RaftOptions;
+import org.apache.ignite.raft.jraft.option.ReadOnlyOption;
 import org.apache.ignite.raft.jraft.rpc.RpcClientEx;
 import org.apache.ignite.raft.jraft.rpc.RpcRequests;
 import org.apache.ignite.raft.jraft.rpc.RpcServer;
@@ -3332,6 +3334,62 @@ public class ItNodeTest {
         LOG.info("Elect new leader is {}, curTerm={}", leader.getLeaderId(), ((NodeImpl) leader).getCurrentTerm());
     }
 
+    /**
+     * Tests if a read using leader leases works correctly after previous leader segmentation.
+     */
+    @Test
+    public void testLeaseReadAfterSegmentation() throws Exception {
+        List<PeerId> peers = TestUtils.generatePeers(3);
+        cluster = new TestCluster("unittest", dataPath, peers, 3_000, testInfo);
+
+        for (PeerId peer : peers) {
+            RaftOptions opts = new RaftOptions();
+            opts.setElectionHeartbeatFactor(2); // Election timeout divisor.
+            opts.setReadOnlyOptions(ReadOnlyOption.ReadOnlyLeaseBased);
+            assertTrue(cluster.start(peer.getEndpoint(), false, 300, false, null, opts));
+        }
+
+        cluster.waitLeader();
+
+        NodeImpl leader = (NodeImpl) cluster.getLeader();
+        assertNotNull(leader);
+        cluster.ensureLeader(leader);
+
+        sendTestTaskAndWait(leader);
+        cluster.ensureSame();
+
+        DefaultRaftClientService rpcService = (DefaultRaftClientService) leader.getRpcClientService();
+        RpcClientEx rpcClientEx = (RpcClientEx) rpcService.getRpcClient();
+
+        AtomicInteger cnt = new AtomicInteger();
+
+        rpcClientEx.blockMessages((msg, nodeId) -> {
+            assertTrue(msg instanceof RpcRequests.AppendEntriesRequest);
+
+            if (cnt.get() >= 2)
+                return true;
+
+            LOG.info("Send heartbeat: " + msg + " to " + nodeId);
+
+            cnt.incrementAndGet();
+
+            return false;
+        });
+
+        assertTrue(waitForCondition(() -> cluster.getLeader() != null &&
+            !leader.getNodeId().equals(cluster.getLeader().getNodeId()), 10_000));
+
+        CompletableFuture<Status> res = new CompletableFuture<>();
+
+        cluster.getLeader().readIndex(null, new ReadIndexClosure() {
+            @Override public void run(Status status, long index, byte[] reqCtx) {
+                res.complete(status);
+            }
+        });
+
+        assertTrue(res.get().isOk());
+    }
+
     private NodeOptions createNodeOptions() {
         NodeOptions options = new NodeOptions();
 
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
index 0ed93ff..cf4de99 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/CounterListener.java
@@ -21,17 +21,20 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Consumer;
 import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.apache.ignite.raft.jraft.util.ExecutorServiceHelper;
 import org.apache.ignite.raft.jraft.util.Utils;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * The counter listener implementation.
@@ -113,6 +116,11 @@ public class CounterListener implements RaftGroupListener {
         ExecutorServiceHelper.shutdownAndAwaitTermination(executor);
     }
 
+    /** {@inheritDoc} */
+    @Override public @Nullable CompletableFuture<Void> onBeforeApply(Command command) {
+        return null;
+    }
+
     /**
      * Returns current value.
      */
diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ItJraftCounterServerTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ItJraftCounterServerTest.java
index d8ce9c6..4d9124d 100644
--- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ItJraftCounterServerTest.java
+++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/server/ItJraftCounterServerTest.java
@@ -305,10 +305,10 @@ class ItJraftCounterServerTest extends RaftServerAbstractTest {
     @NotNull
     private Set<Thread> getAllDisruptorCurrentThreads() {
         return Thread.getAllStackTraces().keySet().stream().filter(t ->
-                        t.getName().contains("JRaft-FSMCaller-Disruptor")
-                                || t.getName().contains("JRaft-NodeImpl-Disruptor")
-                                || t.getName().contains("JRaft-ReadOnlyService-Disruptor")
-                                || t.getName().contains("JRaft-LogManager-Disruptor"))
+                t.getName().contains("JRaft-FSMCaller-Disruptor")
+                        || t.getName().contains("JRaft-NodeImpl-Disruptor")
+                        || t.getName().contains("JRaft-ReadOnlyService-Disruptor")
+                        || t.getName().contains("JRaft-LogManager-Disruptor"))
                 .collect(Collectors.toSet());
     }
 
@@ -568,13 +568,16 @@ class ItJraftCounterServerTest extends RaftServerAbstractTest {
                 super.onWrite(wrapper);
             }
 
-            @Override public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
+            @Override
+            public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
                 Iterator<CommandClosure<ReadCommand>> wrapper = new Iterator<>() {
-                    @Override public boolean hasNext() {
+                    @Override
+                    public boolean hasNext() {
                         return iterator.hasNext();
                     }
 
-                    @Override public CommandClosure<ReadCommand> next() {
+                    @Override
+                    public CommandClosure<ReadCommand> next() {
                         CommandClosure<ReadCommand> cmd = iterator.next();
 
                         cmd.result(new RuntimeException("Another expected message"));
@@ -743,8 +746,8 @@ class ItJraftCounterServerTest extends RaftServerAbstractTest {
      * Applies increments.
      *
      * @param client The client
-     * @param start  Start element.
-     * @param stop   Stop element.
+     * @param start Start element.
+     * @param stop Stop element.
      * @return The counter value.
      * @throws Exception If failed.
      */
@@ -774,8 +777,8 @@ class ItJraftCounterServerTest extends RaftServerAbstractTest {
      * Validates state machine.
      *
      * @param expected Expected value.
-     * @param server   The server.
-     * @param groupId  Group id.
+     * @param server The server.
+     * @param groupId Group id.
      * @return Validation result.
      */
     private static boolean validateStateMachine(long expected, JraftServerImpl server, String groupId) {
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
index 88c87de..b525088 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.raft;
 import java.nio.file.Path;
 import java.util.Collection;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -42,6 +43,7 @@ import org.apache.ignite.raft.jraft.RaftMessagesFactory;
 import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl;
 import org.apache.ignite.raft.jraft.util.Utils;
 import org.jetbrains.annotations.ApiStatus.Experimental;
+import org.jetbrains.annotations.TestOnly;
 
 /**
  * Best raft manager ever since 1982.
@@ -53,9 +55,7 @@ public class Loza implements IgniteComponent {
     /** Raft client pool name. */
     public static final String CLIENT_POOL_NAME = "Raft-Group-Client";
 
-    /**
-     * Raft client pool size. Size was taken from jraft's TimeManager.
-     */
+    /** Raft client pool size. Size was taken from jraft's TimeManager. */
     private static final int CLIENT_POOL_SIZE = Math.min(Utils.cpus() * 3, 20);
 
     /** Timeout. */
@@ -79,9 +79,10 @@ public class Loza implements IgniteComponent {
     private final ScheduledExecutorService executor;
 
     /**
-     * Constructor.
+     * The constructor.
      *
      * @param clusterNetSvc Cluster network service.
+     * @param dataPath      Data path.
      */
     public Loza(ClusterService clusterNetSvc, Path dataPath) {
         this.clusterNetSvc = clusterNetSvc;
@@ -95,6 +96,24 @@ public class Loza implements IgniteComponent {
         );
     }
 
+    /**
+     * The constructor. Used for testing purposes.
+     *
+     * @param srv Pre-started raft server.
+     */
+    @TestOnly
+    public Loza(JraftServerImpl srv) {
+        this.clusterNetSvc = srv.clusterService();
+
+        this.raftServer = srv;
+
+        this.executor = new ScheduledThreadPoolExecutor(CLIENT_POOL_SIZE,
+                new NamedThreadFactory(NamedThreadFactory.threadPrefix(clusterNetSvc.localConfiguration().getName(),
+                        CLIENT_POOL_NAME)
+                )
+        );
+    }
+
     /** {@inheritDoc} */
     @Override
     public void start() {
@@ -238,4 +257,34 @@ public class Loza implements IgniteComponent {
     public void stopRaftGroup(String groupId) {
         raftServer.stopRaftGroup(groupId);
     }
+
+    /**
+     * Returns a cluster service.
+     *
+     * @return An underlying network service.
+     */
+    @TestOnly
+    public ClusterService service() {
+        return clusterNetSvc;
+    }
+
+    /**
+     * Returns a raft server.
+     *
+     * @return An underlying raft server.
+     */
+    @TestOnly
+    public RaftServer server() {
+        return raftServer;
+    }
+
+    /**
+     * Returns started groups.
+     *
+     * @return Started groups.
+     */
+    @TestOnly
+    public Set<String> startedGroups() {
+        return raftServer.startedGroups();
+    }
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
index 8890599..526cc4d 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java
@@ -18,11 +18,13 @@
 package org.apache.ignite.internal.raft.server;
 
 import java.util.List;
+import java.util.Set;
 import org.apache.ignite.internal.manager.IgniteComponent;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.jetbrains.annotations.Nullable;
+import org.jetbrains.annotations.TestOnly;
 
 /**
  * The RAFT protocol based replication server. Supports multiple RAFT groups. The server listens for client commands, submits them to a
@@ -60,4 +62,12 @@ public interface RaftServer extends IgniteComponent {
      * @return Local peer or null if the group is not started.
      */
     @Nullable Peer localPeer(String groupId);
+
+    /**
+     * Returns a set of started partition groups.
+     *
+     * @return Started groups.
+     */
+    @TestOnly
+    Set<String> startedGroups();
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java
index 90ac87d..e33fd47 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java
@@ -26,6 +26,7 @@ import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
@@ -92,7 +93,7 @@ public class JraftServerImpl implements RaftServer {
     private ExecutorService requestExecutor;
 
     /**
-     * Constructor.
+     * The constructor.
      *
      * @param service  Cluster service.
      * @param dataPath Data path.
@@ -102,17 +103,13 @@ public class JraftServerImpl implements RaftServer {
     }
 
     /**
-     * Constructor.
+     * The constructor.
      *
      * @param service  Cluster service.
      * @param dataPath Data path.
      * @param opts     Default node options.
      */
-    public JraftServerImpl(
-            ClusterService service,
-            Path dataPath,
-            NodeOptions opts
-    ) {
+    public JraftServerImpl(ClusterService service, Path dataPath, NodeOptions opts) {
         this.service = service;
         this.dataPath = dataPath;
         this.nodeManager = new NodeManager();
@@ -250,8 +247,7 @@ public class JraftServerImpl implements RaftServer {
 
     /** {@inheritDoc} */
     @Override
-    public synchronized boolean startRaftGroup(String groupId, RaftGroupListener lsnr,
-            @Nullable List<Peer> initialConf) {
+    public synchronized boolean startRaftGroup(String groupId, RaftGroupListener lsnr, @Nullable List<Peer> initialConf) {
         if (groups.containsKey(groupId)) {
             return false;
         }
@@ -334,6 +330,12 @@ public class JraftServerImpl implements RaftServer {
         return groups.get(groupId);
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public Set<String> startedGroups() {
+        return groups.keySet();
+    }
+
     /**
      * Wrapper of {@link StateMachineAdapter}.
      */
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java
index ead9a21..4fce934 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java
@@ -1538,11 +1538,10 @@ public class NodeImpl implements Node, RaftServerService {
         final long lastCommittedIndex = this.ballotBox.getLastCommittedIndex();
         if (this.logManager.getTerm(lastCommittedIndex) != this.currTerm) {
             // Reject read only request when this leader has not committed any log entry at its term
-            closure
-                .run(new Status(
-                    RaftError.EAGAIN,
-                    "ReadIndex request rejected because leader has not committed any log entry at its term, logIndex=%d, currTerm=%d.",
-                    lastCommittedIndex, this.currTerm));
+            closure.run(new Status(
+                RaftError.EAGAIN,
+                "ReadIndex request rejected because leader has not committed any log entry at its term, logIndex=%d, currTerm=%d.",
+                lastCommittedIndex, this.currTerm));
             return;
         }
         respBuilder.index(lastCommittedIndex);
@@ -1707,9 +1706,11 @@ public class NodeImpl implements Node, RaftServerService {
     // in read_lock
     private boolean isLeaderLeaseValid() {
         final long monotonicNowMs = Utils.monotonicMs();
+        // Test with a current start lease timestamp.
         if (checkLeaderLease(monotonicNowMs)) {
             return true;
         }
+        // Refresh start lease timestamp and try again.
         checkDeadNodes0(this.conf.getConf().getPeers(), monotonicNowMs, false, null);
         return checkLeaderLease(monotonicNowMs);
     }
@@ -1719,7 +1720,7 @@ public class NodeImpl implements Node, RaftServerService {
     }
 
     private boolean isCurrentLeaderValid() {
-        return Utils.monotonicMs() - this.lastLeaderTimestamp < this.options.getElectionTimeoutMs();
+        return checkLeaderLease(Utils.monotonicMs());
     }
 
     private void updateLastLeaderTimestamp(final long lastLeaderTimestamp) {
@@ -1770,6 +1771,7 @@ public class NodeImpl implements Node, RaftServerService {
                         request.serverId(), request.term(), this.currTerm);
                     break;
                 }
+
                 doUnlock = false;
                 this.writeLock.unlock();
 
@@ -1777,6 +1779,7 @@ public class NodeImpl implements Node, RaftServerService {
 
                 doUnlock = true;
                 this.writeLock.lock();
+
                 // vote need ABA check after unlock&writeLock
                 if (request.term() != this.currTerm) {
                     LOG.warn("Node {} raise term {} when get lastLogId.", getNodeId(), this.currTerm);
@@ -2198,7 +2201,6 @@ public class NodeImpl implements Node, RaftServerService {
     }
 
     /**
-     * TODO asch https://issues.apache.org/jira/browse/IGNITE-14843
      * @param peers Peers list.
      * @param monotonicNowMs The timestamp.
      * @param checkReplicator {@code True} to check replicator.
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/disruptor/StripedDisruptor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/disruptor/StripedDisruptor.java
index 4269a93..2a95fea 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/disruptor/StripedDisruptor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/disruptor/StripedDisruptor.java
@@ -224,14 +224,14 @@ public class StripedDisruptor<T extends GroupAware> {
         private final String name;
 
         /** There are exception handlers per group. */
-        private final ConcurrentHashMap<String, BiConsumer<T, Throwable>> subscrivers;
+        private final ConcurrentHashMap<String, BiConsumer<T, Throwable>> subscribers;
 
         /**
          * @param name Name of the Disruptor instance.
          */
         StripeExceptionHandler(String name) {
             this.name = name;
-            this.subscrivers = new ConcurrentHashMap<>();
+            this.subscribers = new ConcurrentHashMap<>();
         }
 
         /**
@@ -241,7 +241,7 @@ public class StripedDisruptor<T extends GroupAware> {
          * @param handler Exception handler.
          */
         void subscribe(String group, BiConsumer<T, Throwable> handler) {
-            subscrivers.put(group, handler);
+            subscribers.put(group, handler);
         }
 
         /**
@@ -250,7 +250,7 @@ public class StripedDisruptor<T extends GroupAware> {
          * @param group Group id.
          */
         void unsubscribe(String group) {
-            subscrivers.remove(group);
+            subscribers.remove(group);
         }
 
         /** {@inheritDoc} */
@@ -266,7 +266,7 @@ public class StripedDisruptor<T extends GroupAware> {
 
         /** {@inheritDoc} */
         @Override public void handleEventException(Throwable ex, long sequence, T event) {
-            BiConsumer<T, Throwable> handler = subscrivers.get(event.groupId());
+            BiConsumer<T, Throwable> handler = subscribers.get(event.groupId());
 
             LOG.error("Handle disruptor event error [name={}, event={}, hasHandler={}]", ex, name, event, handler != null);
 
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java
index 8b09b02..5858379 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java
@@ -231,7 +231,7 @@ public enum RaftError {
     EIO(1014),
 
     /**
-     * Invalid value.
+     * Invalid value
      */
     EINVAL(1015),
 
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/NodeOptions.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/NodeOptions.java
index a3e306a..632446e 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/NodeOptions.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/NodeOptions.java
@@ -351,6 +351,7 @@ public class NodeOptions extends RpcOptions implements Copiable<NodeOptions> {
     }
 
     public int getLeaderLeaseTimeoutMs() {
+        // TODO asch precompute IGNITE-14832
         return this.electionTimeoutMs * this.leaderLeaseTimeRatio / 100;
     }
 
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/ReadOnlyOption.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/ReadOnlyOption.java
index 3fba00d..ab20b4c 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/ReadOnlyOption.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/ReadOnlyOption.java
@@ -20,7 +20,6 @@ package org.apache.ignite.raft.jraft.option;
  * Read only options.
  */
 public enum ReadOnlyOption {
-
     // ReadOnlySafe guarantees the linearizability of the read only request by
     // communicating with the quorum. It is the default and suggested option.
     ReadOnlySafe,
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/RpcResponseClosure.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/RpcResponseClosure.java
index bc8eae4..84ca83b 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/RpcResponseClosure.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/RpcResponseClosure.java
@@ -21,9 +21,6 @@ import org.apache.ignite.raft.jraft.Closure;
 /**
  * RPC response closure.
  *
- *
- * 2018-Apr-08 5:55:01 PM
- *
  * @param <T>
  */
 public interface RpcResponseClosure<T extends Message> extends Closure {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/ActionRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/ActionRequestProcessor.java
index 8064f2d..0800e0c 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/ActionRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/ActionRequestProcessor.java
@@ -19,7 +19,9 @@ package org.apache.ignite.raft.jraft.rpc.impl;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.raft.client.Command;
@@ -59,7 +61,8 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
     }
 
     /** {@inheritDoc} */
-    @Override public void handleRequest(RpcContext rpcCtx, ActionRequest request) {
+    @Override
+    public void handleRequest(RpcContext rpcCtx, ActionRequest request) {
         Node node = rpcCtx.getNodeManager().get(request.groupId(), new PeerId(rpcCtx.getLocalAddress()));
 
         if (node == null) {
@@ -68,10 +71,48 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
             return;
         }
 
-        if (request.command() instanceof WriteCommand) {
-            node.apply(new Task(ByteBuffer.wrap(JDKMarshaller.DEFAULT.marshall(request.command())),
+        JraftServerImpl.DelegatingStateMachine fsm = (JraftServerImpl.DelegatingStateMachine) node.getOptions().getFsm();
+
+        // Apply a filter before commiting to STM.
+        CompletableFuture<Void> fut = fsm.getListener().onBeforeApply(request.command());
+
+        if (fut != null) {
+            fut.handle(new BiFunction<Void, Throwable, Void>() {
+                @Override
+                public Void apply(Void ignored, Throwable err) {
+                    if (err == null) {
+                        if (request.command() instanceof WriteCommand) {
+                            applyWrite(node, request, rpcCtx);
+                        } else {
+                            applyRead(node, request, rpcCtx);
+                        }
+                    } else {
+                        sendSMError(rpcCtx, err, false);
+                    }
+
+                    return null;
+                }
+            });
+        } else {
+            if (request.command() instanceof WriteCommand) {
+                applyWrite(node, request, rpcCtx);
+            } else {
+                applyRead(node, request, rpcCtx);
+            }
+        }
+    }
+
+    /**
+     * @param node    The node.
+     * @param request The request.
+     * @param rpcCtx  The context.
+     */
+    private void applyWrite(Node node, ActionRequest request, RpcContext rpcCtx) {
+        // TODO asch get rid of JDK marshaller IGNITE-14832
+        node.apply(new Task(ByteBuffer.wrap(JDKMarshaller.DEFAULT.marshall(request.command())),
                 new CommandClosureImpl<>(request.command()) {
-                    @Override public void result(Serializable res) {
+                    @Override
+                    public void result(Serializable res) {
                         if (res instanceof Throwable) {
                             sendSMError(rpcCtx, (Throwable)res, true);
 
@@ -81,71 +122,76 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
                         rpcCtx.sendResponse(factory.actionResponse().result(res).build());
                     }
 
-                    @Override public void run(Status status) {
+                    @Override
+                    public void run(Status status) {
                         assert !status.isOk() : status;
 
                         sendRaftError(rpcCtx, status, node);
                     }
                 }));
-        }
-        else {
-            if (request.readOnlySafe()) {
-                node.readIndex(BytesUtil.EMPTY_BYTES, new ReadIndexClosure() {
-                    @Override public void run(Status status, long index, byte[] reqCtx) {
-                        if (status.isOk()) {
-                            JraftServerImpl.DelegatingStateMachine fsm =
-                                (JraftServerImpl.DelegatingStateMachine) node.getOptions().getFsm();
+    }
 
-                            try {
-                                fsm.getListener().onRead(List.<CommandClosure<ReadCommand>>of(new CommandClosure<>() {
-                                    @Override public ReadCommand command() {
-                                        return (ReadCommand)request.command();
-                                    }
+    /**
+     * @param node    The node.
+     * @param request The request.
+     * @param rpcCtx  The context.
+     */
+    private void applyRead(Node node, ActionRequest request, RpcContext rpcCtx) {
+        if (request.readOnlySafe()) {
+            node.readIndex(BytesUtil.EMPTY_BYTES, new ReadIndexClosure() {
+                @Override public void run(Status status, long index, byte[] reqCtx) {
+                    if (status.isOk()) {
+                        JraftServerImpl.DelegatingStateMachine fsm =
+                                (JraftServerImpl.DelegatingStateMachine) node.getOptions().getFsm();
 
-                                    @Override public void result(Serializable res) {
-                                        if (res instanceof Throwable) {
-                                            sendSMError(rpcCtx, (Throwable)res, true);
+                        try {
+                            fsm.getListener().onRead(List.<CommandClosure<ReadCommand>>of(new CommandClosure<>() {
+                                @Override public ReadCommand command() {
+                                    return (ReadCommand)request.command();
+                                }
 
-                                            return;
-                                        }
+                                @Override public void result(Serializable res) {
+                                    if (res instanceof Throwable) {
+                                        sendSMError(rpcCtx, (Throwable)res, true);
 
-                                        rpcCtx.sendResponse(factory.actionResponse().result(res).build());
+                                        return;
                                     }
-                                }).iterator());
-                            }
-                            catch (Exception e) {
-                                sendRaftError(rpcCtx, RaftError.ESTATEMACHINE, e.getMessage());
-                            }
+
+                                    rpcCtx.sendResponse(factory.actionResponse().result(res).build());
+                                }
+                            }).iterator());
+                        }
+                        catch (Exception e) {
+                            sendRaftError(rpcCtx, RaftError.ESTATEMACHINE, e.getMessage());
                         }
-                        else
-                            sendRaftError(rpcCtx, status, node);
                     }
-                });
-            }
-            else {
-                // TODO asch remove copy paste, batching https://issues.apache.org/jira/browse/IGNITE-14832
-                JraftServerImpl.DelegatingStateMachine fsm =
+                    else
+                        sendRaftError(rpcCtx, status, node);
+                }
+            });
+        } else {
+            // TODO asch remove copy paste, batching https://issues.apache.org/jira/browse/IGNITE-14832
+            JraftServerImpl.DelegatingStateMachine fsm =
                     (JraftServerImpl.DelegatingStateMachine) node.getOptions().getFsm();
 
-                try {
-                    fsm.getListener().onRead(List.<CommandClosure<ReadCommand>>of(new CommandClosure<>() {
-                        @Override public ReadCommand command() {
-                            return (ReadCommand)request.command();
-                        }
+            try {
+                fsm.getListener().onRead(List.<CommandClosure<ReadCommand>>of(new CommandClosure<>() {
+                    @Override public ReadCommand command() {
+                        return (ReadCommand)request.command();
+                    }
 
-                        @Override public void result(Serializable res) {
-                            if (res instanceof Throwable) {
-                                sendSMError(rpcCtx, (Throwable)res, true);
+                    @Override public void result(Serializable res) {
+                        if (res instanceof Throwable) {
+                            sendSMError(rpcCtx, (Throwable)res, true);
 
-                                return;
-                            }
-                            rpcCtx.sendResponse(factory.actionResponse().result(res).build());
+                            return;
                         }
-                    }).iterator());
-                }
-                catch (Exception e) {
-                    sendRaftError(rpcCtx, RaftError.ESTATEMACHINE, e.getMessage());
-                }
+                        rpcCtx.sendResponse(factory.actionResponse().result(res).build());
+                    }
+                }).iterator());
+            }
+            catch (Exception e) {
+                sendRaftError(rpcCtx, RaftError.ESTATEMACHINE, e.getMessage());
             }
         }
     }
@@ -163,9 +209,9 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
     /**
      * Sends raft error response with raft error code and message.
      *
-     * @param ctx Context.
+     * @param ctx   Context.
      * @param error RaftError code.
-     * @param msg Message.
+     * @param msg   Message.
      */
     private void sendRaftError(RpcContext ctx, RaftError error, String msg) {
         RpcRequests.ErrorResponse resp = factory.errorResponse()
@@ -179,8 +225,8 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
     /**
      * Sends client's state machine error response with passed throwable.
      *
-     * @param ctx Context.
-     * @param th Throwable that must be passes to response.
+     * @param ctx       The context.
+     * @param th        Throwable that must be passes to response.
      * @param compacted {@code true} if throwable must be changed to compacted version of throwable.
      * See {@link SMCompactedThrowable}
      */
@@ -195,9 +241,9 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
     }
 
     /**
-     * @param ctx The context.
+     * @param ctx    The context.
      * @param status The status.
-     * @param node Raft node.
+     * @param node   Raft node.
      */
     private void sendRaftError(RpcContext ctx, Status status, Node node) {
         RaftError raftError = status.getRaftError();
@@ -214,19 +260,14 @@ public class ActionRequestProcessor implements RpcProcessor<ActionRequest> {
         ctx.sendResponse(response);
     }
 
-    /**
-     *
-     */
+    /** The implementation. */
     private abstract static class CommandClosureImpl<T extends Command> implements Closure, CommandClosure<T> {
-        /**
-         *
-         */
         private final T command;
 
         /**
          * @param command The command.
          */
-        CommandClosureImpl(T command) {
+        public CommandClosureImpl(T command) {
             this.command = command;
         }
 
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcServer.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcServer.java
index 164f4b3..2a39de3 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcServer.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/IgniteRpcServer.java
@@ -21,6 +21,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executor;
+import java.util.concurrent.RejectedExecutionException;
 import org.apache.ignite.network.NetworkMessageHandler;
 import org.apache.ignite.raft.jraft.RaftMessageGroup;
 import org.apache.ignite.raft.jraft.RaftMessagesFactory;
@@ -66,6 +67,7 @@ public class IgniteRpcServer implements RpcServer<Void> {
     private final Map<String, RpcProcessor> processors = new ConcurrentHashMap<>();
 
     /**
+     * @param lockManager The lock manager.
      * @param service The cluster service.
      * @param nodeManager The node manager.
      * @param raftMessagesFactory Message factory.
@@ -160,27 +162,31 @@ public class IgniteRpcServer implements RpcServer<Void> {
 
             RpcProcessor<NetworkMessage> finalPrc = prc;
 
-            executor.execute(() -> {
-                var context = new RpcContext() {
-                    @Override public NodeManager getNodeManager() {
-                        return nodeManager;
-                    }
-
-                    @Override public void sendResponse(Object responseObj) {
-                        service.messagingService().send(senderAddr, (NetworkMessage) responseObj, correlationId);
-                    }
-
-                    @Override public NetworkAddress getRemoteAddress() {
-                        return senderAddr;
-                    }
-
-                    @Override public NetworkAddress getLocalAddress() {
-                        return service.topologyService().localMember().address();
-                    }
-                };
-
-                finalPrc.handleRequest(context, message);
-            });
+            try {
+                executor.execute(() -> {
+                    var context = new RpcContext() {
+                        @Override public NodeManager getNodeManager() {
+                            return nodeManager;
+                        }
+
+                        @Override public void sendResponse(Object responseObj) {
+                            service.messagingService().send(senderAddr, (NetworkMessage) responseObj, correlationId);
+                        }
+
+                        @Override public NetworkAddress getRemoteAddress() {
+                            return senderAddr;
+                        }
+
+                        @Override public NetworkAddress getLocalAddress() {
+                            return service.topologyService().localMember().address();
+                        }
+                    };
+
+                    finalPrc.handleRequest(context, message);
+                });
+            } catch (RejectedExecutionException e) {
+                // Node is stopping.
+            }
         }
     }
 
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
index 10f719f..0711b2d 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
@@ -41,12 +41,14 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
+import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.network.ClusterService;
@@ -168,7 +170,7 @@ public class RaftGroupServiceImpl implements RaftGroupService {
 
         return service.refreshLeader().handle((unused, throwable) -> {
             if (throwable != null)
-                LOG.error("Failed to refresh a leader", throwable);
+                LOG.error("Failed to refresh a leader [groupId={}]", throwable, groupId);
 
             return service;
         });
@@ -464,14 +466,19 @@ public class RaftGroupServiceImpl implements RaftGroupService {
     @Override public <R> CompletableFuture<R> run(Peer peer, ReadCommand cmd) {
         ActionRequest req = factory.actionRequest().command(cmd).groupId(groupId).readOnlySafe(false).build();
 
-        CompletableFuture<?> fut = cluster.messagingService().invoke(peer.address(), req, timeout);
+        CompletableFuture<ActionResponse> fut = cluster.messagingService().invoke(peer.address(), req, timeout);
 
-        return fut.thenApply(resp -> (R) ((ActionResponse) resp).result());
+        return fut.thenApply(resp -> (R) resp.result());
     }
 
     /** {@inheritDoc} */
     @Override public void shutdown() {
+        // No-op.
+    }
 
+    /** {@inheritDoc} */
+    @Override public ClusterService clusterService() {
+        return cluster;
     }
 
     /**
@@ -484,6 +491,14 @@ public class RaftGroupServiceImpl implements RaftGroupService {
      * @param <R> Response type.
      */
     private <R> void sendWithRetry(Peer peer, Object req, long stopTime, CompletableFuture<R> fut) {
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("sendWithRetry peers={} req={} from={} to={}",
+                    peers,
+                    S.toString(req),
+                    cluster.topologyService().localMember().address(),
+                    peer.address());
+        }
+        
         if (currentTimeMillis() >= stopTime) {
             fut.completeExceptionally(new TimeoutException());
 
@@ -495,6 +510,14 @@ public class RaftGroupServiceImpl implements RaftGroupService {
         //TODO: IGNITE-15389 org.apache.ignite.internal.metastorage.client.CursorImpl has potential deadlock inside
         fut0.whenCompleteAsync(new BiConsumer<Object, Throwable>() {
             @Override public void accept(Object resp, Throwable err) {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("sendWithRetry resp={} from={} to={} err={}",
+                            S.toString(resp),
+                            cluster.topologyService().localMember().address(),
+                            peer.address(),
+                            err == null ? null : err.getMessage());
+                }
+                
                 if (err != null) {
                     if (recoverable(err)) {
                         executor.schedule(() -> {
@@ -503,19 +526,21 @@ public class RaftGroupServiceImpl implements RaftGroupService {
                             return null;
                         }, retryDelay, TimeUnit.MILLISECONDS);
                     }
-                    else
+                    else {
                         fut.completeExceptionally(err);
+                    }
                 }
                 else if (resp instanceof RpcRequests.ErrorResponse) {
                     RpcRequests.ErrorResponse resp0 = (RpcRequests.ErrorResponse) resp;
 
                     if (resp0.errorCode() == RaftError.SUCCESS.getNumber()) { // Handle OK response.
                         leader = peer; // The OK response was received from a leader.
-
+                        
                         fut.complete(null); // Void response.
                     }
                     else if (resp0.errorCode() == RaftError.EBUSY.getNumber() ||
-                        resp0.errorCode() == (RaftError.EAGAIN.getNumber())) {
+                        resp0.errorCode() == (RaftError.EAGAIN.getNumber()) ||
+                        resp0.errorCode() == (RaftError.ENOENT.getNumber())) { // Possibly a node has not been started.
                         executor.schedule(() -> {
                             sendWithRetry(peer, req, stopTime, fut);
 
@@ -535,18 +560,18 @@ public class RaftGroupServiceImpl implements RaftGroupService {
                         }
                         else {
                             leader = parsePeer(resp0.leaderId()); // Update a leader.
-
+                            
                             executor.schedule(() -> {
                                 sendWithRetry(leader, req, stopTime, fut);
 
                                 return null;
                             }, retryDelay, TimeUnit.MILLISECONDS);
-
                         }
                     }
-                    else
+                    else {
                         fut.completeExceptionally(
                             new RaftException(RaftError.forNumber(resp0.errorCode()), resp0.errorMsg()));
+                    }
                 }
                 else if (resp instanceof RpcRequests.SMErrorResponse) {
                     SMThrowable th = ((RpcRequests.SMErrorResponse)resp).error();
@@ -586,7 +611,7 @@ public class RaftGroupServiceImpl implements RaftGroupService {
      * @return {@code True} if this is a recoverable exception.
      */
     private boolean recoverable(Throwable t) {
-        return t.getCause() instanceof IOException;
+        return t.getCause() instanceof IOException || t.getCause() instanceof TimeoutException;
     }
 
     /**
@@ -595,8 +620,7 @@ public class RaftGroupServiceImpl implements RaftGroupService {
     private Peer randomNode() {
         List<Peer> peers0 = peers;
 
-        if (peers0 == null || peers0.isEmpty())
-            return null;
+        assert peers0 != null && !peers0.isEmpty();
 
         return peers0.get(current().nextInt(peers0.size()));
     }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java
index 7799da6..61b4fd7 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java
@@ -29,8 +29,6 @@ import org.apache.ignite.raft.jraft.rpc.RaftRpcFactory;
 
 /**
  * AddLearners request processor.
- *
- * @author jiachun.fjc
  */
 public class AddLearnersRequestProcessor extends BaseCliRequestProcessor<AddLearnersRequest> {
     public AddLearnersRequestProcessor(Executor executor, RaftMessagesFactory msgFactory) {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/BaseCliRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/BaseCliRequestProcessor.java
index fd6308a..e33457a 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/BaseCliRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/cli/BaseCliRequestProcessor.java
@@ -34,8 +34,6 @@ import org.apache.ignite.raft.jraft.util.StringUtils;
 /**
  * Base template to handle cli requests.
  *
- * 2018-Apr-09 11:51:42 AM
- *
  * @param <T>
  */
 public abstract class BaseCliRequestProcessor<T extends Message> extends RpcRequestProcessor<T> {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/NodeRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/NodeRequestProcessor.java
index b72c404..2aef9a7 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/NodeRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/NodeRequestProcessor.java
@@ -31,7 +31,6 @@ import org.apache.ignite.raft.jraft.rpc.RpcRequestProcessor;
  * Node handle requests processor template.
  *
  * @param <T> Message
- * @author jiachun.fjc
  */
 public abstract class NodeRequestProcessor<T extends Message> extends RpcRequestProcessor<T> {
     public NodeRequestProcessor(Executor executor, RaftMessagesFactory msgFactory) {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/ReadIndexRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/ReadIndexRequestProcessor.java
index 4938481..c4a1551 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/ReadIndexRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/ReadIndexRequestProcessor.java
@@ -28,11 +28,8 @@ import org.apache.ignite.raft.jraft.rpc.RpcResponseClosureAdapter;
 
 /**
  * Handle read index request.
- *
- * @author jiachun.fjc
  */
 public class ReadIndexRequestProcessor extends NodeRequestProcessor<ReadIndexRequest> {
-
     public ReadIndexRequestProcessor(Executor executor, RaftMessagesFactory msgFactory) {
         super(executor, msgFactory);
     }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java
index 4dd4b90..0b4030d 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java
@@ -25,11 +25,8 @@ import org.apache.ignite.raft.jraft.rpc.RpcRequests.TimeoutNowRequest;
 
 /**
  * TimeoutNow request processor.
- *
- * @author jiachun.fjc
  */
 public class TimeoutNowRequestProcessor extends NodeRequestProcessor<TimeoutNowRequest> {
-
     public TimeoutNowRequestProcessor(Executor executor, RaftMessagesFactory msgFactory) {
         super(executor, msgFactory);
     }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/Storage.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/Storage.java
index 9780421..275d10c 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/Storage.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/Storage.java
@@ -18,9 +18,6 @@ package org.apache.ignite.raft.jraft.storage;
 
 /**
  * Common interface for storage.
- *
- *
- * 2018-Mar-12 3:52:36 PM
  */
 public interface Storage {
 }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java
index 2f0aa77..4593058 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java
@@ -205,6 +205,9 @@ public class LogManagerImpl implements LogManager {
     }
 
     private void stopDiskThread() {
+        if (this.diskQueue == null)
+            return; // Was not started.
+        
         this.shutDownLatch = new CountDownLatch(1);
         Utils.runInThread(nodeOptions.getCommonExecutor(), () -> this.diskQueue.publishEvent((event, sequence) -> {
             event.reset();
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/ArrayDeque.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/ArrayDeque.java
index 9f9a59a..98c37c4 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/ArrayDeque.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/ArrayDeque.java
@@ -21,9 +21,6 @@ import java.util.List;
 /**
  * Extend array list to add peek/poll first/last element.
  *
- *
- * 2018-Apr-11 11:14:38 AM
- *
  * @param <E>
  */
 public class ArrayDeque<E> extends java.util.ArrayList<E> {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Bits.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Bits.java
index e021e85..33f02be 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Bits.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Bits.java
@@ -18,11 +18,8 @@ package org.apache.ignite.raft.jraft.util;
 
 /**
  * Bits util.
- *
- * @author jiachun.fjc
  */
 public class Bits {
-
     public static int getInt(final byte[] b, final int off) {
         return HeapByteBufUtil.getInt(b, off);
     }
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Copiable.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Copiable.java
index d9b3596..820cac8 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Copiable.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/Copiable.java
@@ -19,9 +19,6 @@ package org.apache.ignite.raft.jraft.util;
 /**
  * Copiable mark interface.
  *
- *
- * 2018-Apr-03 11:15:34 AM
- *
  * @param <T>
  */
 public interface Copiable<T> {
diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/concurrent/DefaultFixedThreadsExecutorGroupFactory.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/concurrent/DefaultFixedThreadsExecutorGroupFactory.java
index 49bf3d7..104940e 100644
--- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/concurrent/DefaultFixedThreadsExecutorGroupFactory.java
+++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/util/concurrent/DefaultFixedThreadsExecutorGroupFactory.java
@@ -23,7 +23,7 @@ import org.apache.ignite.raft.jraft.util.Requires;
 import org.apache.ignite.raft.jraft.util.Utils;
 
 /**
- *
+ * TODO asch switch default executor to threadpoolexecutor + mpsc blocking queue IGNITE-14832
  */
 public final class DefaultFixedThreadsExecutorGroupFactory implements FixedThreadsExecutorGroupFactory {
     public static final DefaultFixedThreadsExecutorGroupFactory INSTANCE = new DefaultFixedThreadsExecutorGroupFactory();
diff --git a/modules/raft/src/test/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java b/modules/raft/src/test/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
index 009e827..9672e1b 100644
--- a/modules/raft/src/test/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
+++ b/modules/raft/src/test/java/org/apache/ignite/internal/raft/server/impl/RaftServerImpl.java
@@ -21,6 +21,7 @@ import java.io.Serializable;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -192,7 +193,13 @@ public class RaftServerImpl implements RaftServer {
     public @Nullable Peer localPeer(String groupId) {
         return new Peer(service.topologyService().localMember().address());
     }
-
+    
+    /** {@inheritDoc} */
+    @Override
+    public Set<String> startedGroups() {
+        return listeners.keySet();
+    }
+    
     /**
      * Handle action request.
      *
diff --git a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java
index 0d2b7e9..b24c08d 100644
--- a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java
+++ b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/TestCluster.java
@@ -156,7 +156,7 @@ public class TestCluster {
         int electionTimeoutMs,
         TestInfo testInfo
     ) {
-        this(name, dataPath, peers, new LinkedHashSet<>(), ELECTION_TIMEOUT_MILLIS, null, testInfo);
+        this(name, dataPath, peers, new LinkedHashSet<>(), electionTimeoutMs, null, testInfo);
     }
 
     public TestCluster(
@@ -167,7 +167,7 @@ public class TestCluster {
         int electionTimeoutMs,
         TestInfo testInfo
     ) {
-        this(name, dataPath, peers, learners, ELECTION_TIMEOUT_MILLIS, null, testInfo);
+        this(name, dataPath, peers, learners, electionTimeoutMs, null, testInfo);
     }
 
     public TestCluster(
diff --git a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/EndpointTest.java b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/EndpointTest.java
index 2f3b5c3..d1ab4e4 100644
--- a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/EndpointTest.java
+++ b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/EndpointTest.java
@@ -21,7 +21,6 @@ import org.junit.jupiter.api.Test;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class EndpointTest {
-
     @Test
     public void testToStringReset() {
         final Endpoint ep = new Endpoint("192.168.1.1", 8080);
diff --git a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/concurrent/SingleThreadExecutorBenchmark.java b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/concurrent/SingleThreadExecutorBenchmark.java
index cb536c1..c3d05b1 100644
--- a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/concurrent/SingleThreadExecutorBenchmark.java
+++ b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/util/concurrent/SingleThreadExecutorBenchmark.java
@@ -23,11 +23,13 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedTransferQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import io.netty.util.concurrent.DefaultEventExecutor;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.raft.jraft.util.ExecutorServiceHelper;
 import org.apache.ignite.raft.jraft.util.ThreadPoolUtil;
+import org.jctools.queues.MpscBlockingConsumerArrayQueue;
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
 import org.openjdk.jmh.annotations.Mode;
@@ -41,8 +43,6 @@ import org.openjdk.jmh.runner.RunnerException;
 import org.openjdk.jmh.runner.options.Options;
 import org.openjdk.jmh.runner.options.OptionsBuilder;
 
-//import io.netty.util.concurrent.DefaultEventExecutor;
-
 /**
  *
  */
@@ -107,6 +107,21 @@ public class SingleThreadExecutorBenchmark {
     }
 
     @Benchmark
+    public void defaultSingleThreadPollExecutorWithMpscBlockingQueue() throws InterruptedException {
+        ThreadPoolExecutor pool = ThreadPoolUtil.newBuilder() //
+            .coreThreads(1) //
+            .maximumThreads(1) //
+            .poolName("default") //
+            .enableMetric(false) //
+            .workQueue(new MpscBlockingConsumerArrayQueue<>(TIMES)) // TODO asch IGNITE-15997
+            .keepAliveSeconds(60L) //
+            .threadFactory(new NamedThreadFactory("default", true)) //
+            .build();
+
+        execute(new DefaultSingleThreadExecutor(pool));
+    }
+
+    @Benchmark
     public void mpscSingleThreadExecutor() throws InterruptedException {
         execute(new MpscSingleThreadExecutor(TIMES, new NamedThreadFactory("mpsc", true)));
     }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
index 4de678a..1390a50 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
@@ -121,9 +121,9 @@ public class ItDistributedConfigurationPropertiesTest {
                     new MessageSerializationRegistryImpl(),
                     new TestScaleCubeClusterServiceFactory()
             );
-            
+
             raftManager = new Loza(clusterService, workDir);
-            
+
             cfgManager = new ConfigurationManager(
                     List.of(NodeConfiguration.KEY),
                     Map.of(),
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
index eb4f942..c435400 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
@@ -38,8 +38,12 @@ import org.apache.ignite.internal.manager.IgniteComponent;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
 import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.table.distributed.TableTxManagerImpl;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.tx.LockManager;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.impl.HeapLockManager;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.internal.vault.persistence.PersistentVaultService;
 import org.apache.ignite.network.ClusterService;
@@ -67,6 +71,10 @@ public class ItDistributedConfigurationStorageTest {
 
         private final ClusterService clusterService;
 
+        private final LockManager lockManager;
+
+        private final TxManager txManager;
+
         private final Loza raftManager;
 
         private final ConfigurationManager cfgManager;
@@ -74,17 +82,17 @@ public class ItDistributedConfigurationStorageTest {
         private final MetaStorageManager metaStorageManager;
 
         private final DistributedConfigurationStorage cfgStorage;
-        
+
         /**
          * Constructor that simply creates a subset of components of this node.
          */
         Node(TestInfo testInfo, Path workDir) {
             var addr = new NetworkAddress("localhost", 10000);
-            
+
             name = testNodeName(testInfo, addr.port());
-            
+
             vaultManager = new VaultManager(new PersistentVaultService(workDir.resolve("vault")));
-            
+
             clusterService = ClusterServiceTestUtils.clusterService(
                     testInfo,
                     addr.port(),
@@ -92,11 +100,15 @@ public class ItDistributedConfigurationStorageTest {
                     new MessageSerializationRegistryImpl(),
                     new TestScaleCubeClusterServiceFactory()
             );
-            
+
+            lockManager = new HeapLockManager();
+
             raftManager = new Loza(clusterService, workDir);
-            
+
+            txManager = new TableTxManagerImpl(clusterService, lockManager);
+
             List<RootKey<?, ?>> rootKeys = List.of(NodeConfiguration.KEY);
-            
+
             cfgManager = new ConfigurationManager(
                     rootKeys,
                     Map.of(),
@@ -104,7 +116,7 @@ public class ItDistributedConfigurationStorageTest {
                     List.of(),
                     List.of()
             );
-            
+
             metaStorageManager = new MetaStorageManager(
                     vaultManager,
                     cfgManager,
@@ -112,49 +124,49 @@ public class ItDistributedConfigurationStorageTest {
                     raftManager,
                     new SimpleInMemoryKeyValueStorage()
             );
-            
+
             cfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager);
         }
-        
+
         /**
          * Starts the created components.
          */
         void start() throws Exception {
             vaultManager.start();
-            
+
             cfgManager.start();
-            
+
             // metastorage configuration
             var config = String.format("{\"node\": {\"metastorageNodes\": [ \"%s\" ]}}", name);
-            
+
             cfgManager.bootstrap(config);
-            
-            Stream.of(clusterService, raftManager, metaStorageManager).forEach(IgniteComponent::start);
-            
+
+            Stream.of(clusterService, raftManager, txManager, metaStorageManager).forEach(IgniteComponent::start);
+
             // this is needed to avoid assertion errors
             cfgStorage.registerConfigurationListener(changedEntries -> completedFuture(null));
-            
+
             // deploy watches to propagate data from the metastore into the vault
             metaStorageManager.deployWatches();
         }
-        
+
         /**
          * Stops the created components.
          */
         void stop() throws Exception {
             var components =
-                    List.of(metaStorageManager, raftManager, clusterService, cfgManager, vaultManager);
-    
+                    List.of(metaStorageManager, raftManager, txManager, clusterService, cfgManager, vaultManager);
+
             for (IgniteComponent igniteComponent : components) {
                 igniteComponent.beforeNodeStop();
             }
-    
+
             for (IgniteComponent component : components) {
                 component.stop();
             }
         }
     }
-    
+
     /**
      * Tests a scenario when a node is restarted with an existing PDS folder. A node is started and some data is written to the distributed
      * configuration storage. We then expect that the same data can be read by the node after restart.
@@ -164,26 +176,26 @@ public class ItDistributedConfigurationStorageTest {
     @Test
     void testRestartWithPds(@WorkDirectory Path workDir, TestInfo testInfo) throws Exception {
         var node = new Node(testInfo, workDir);
-        
+
         Map<String, Serializable> data = Map.of("foo", "bar");
-        
+
         try {
             node.start();
-            
+
             assertThat(node.cfgStorage.write(data, 0), willBe(equalTo(true)));
-            
+
             waitForCondition(() -> Objects.nonNull(node.vaultManager.get(APPLIED_REV).join().value()), 3000);
         } finally {
             node.stop();
         }
-        
+
         var node2 = new Node(testInfo, workDir);
-        
+
         try {
             node2.start();
-            
+
             Data storageData = node2.cfgStorage.readAll();
-            
+
             assertThat(storageData.values(), equalTo(data));
         } finally {
             node2.stop();
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItNoThreadsLeftTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItNoThreadsLeftTest.java
index 381ddba..a79f192 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItNoThreadsLeftTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItNoThreadsLeftTest.java
@@ -17,8 +17,9 @@
 
 package org.apache.ignite.internal.runner.app;
 
+import static java.util.stream.Collectors.joining;
 import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -83,19 +84,10 @@ public class ItNoThreadsLeftTest extends IgniteAbstractTest {
 
         assertNotNull(tbl);
 
-        Set<Thread> threadsInTime = getCurrentThreads();
-
-        assertTrue(threadsBefore.size() < threadsInTime.size(), threadsBefore.stream()
-                .filter(thread -> !threadsInTime.contains(thread)).map(Thread::getName)
-                .collect(Collectors.joining(",")));
-
         ignite.close();
 
-        Set<Thread> threadsAfter = getCurrentThreads();
-    
-        assertEquals(threadsBefore.size(), threadsAfter.size(), threadsAfter.stream()
-                .filter(thread -> !threadsBefore.contains(thread)).map(Thread::getName)
-                .collect(Collectors.joining(", ")));
+        assertTrue(waitForCondition(() -> threadsBefore.size() == getCurrentThreads().size(), 3_000),
+                getCurrentThreads().stream().filter(thread -> !threadsBefore.contains(thread)).map(Thread::getName).collect(joining(", ")));
     }
 
     /**
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 52ef6f2..bdd66aa 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -58,6 +58,9 @@ import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.schema.configuration.ColumnTypeValidatorImpl;
 import org.apache.ignite.internal.schema.configuration.TableValidatorImpl;
 import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.table.distributed.TableTxManagerImpl;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.impl.HeapLockManager;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.internal.vault.VaultService;
 import org.apache.ignite.internal.vault.persistence.PersistentVaultService;
@@ -109,7 +112,7 @@ public class IgniteImpl implements Ignite {
 
     /** Cluster service (cluster network manager). */
     private final ClusterService clusterSvc;
-    
+
     /** Netty bootstrap factory. */
     private final NettyBootstrapFactory nettyBootstrapFactory;
 
@@ -125,6 +128,9 @@ public class IgniteImpl implements Ignite {
     /** Baseline manager. */
     private final BaselineManager baselineMgr;
 
+    /** Transactions manager. */
+    private final TxManager txManager;
+
     /** Distributed table manager. */
     private final TableManager distributedTblMgr;
 
@@ -166,11 +172,11 @@ public class IgniteImpl implements Ignite {
                 List.of(),
                 List.of()
         );
-    
+
         NetworkConfiguration networkConfiguration = nodeCfgMgr.configurationRegistry().getConfiguration(NetworkConfiguration.KEY);
-    
+
         var clusterLocalConfiguration = new ClusterLocalConfiguration(name, new MessageSerializationRegistryImpl());
-    
+
         nettyBootstrapFactory = new NettyBootstrapFactory(networkConfiguration, clusterLocalConfiguration.getName());
 
         clusterSvc = new ScaleCubeClusterServiceFactory().createClusterService(
@@ -181,6 +187,8 @@ public class IgniteImpl implements Ignite {
 
         raftMgr = new Loza(clusterSvc, workDir);
 
+        txManager = new TableTxManagerImpl(clusterSvc, new HeapLockManager());
+
         metaStorageMgr = new MetaStorageManager(
                 vaultMgr,
                 nodeCfgMgr,
@@ -216,7 +224,8 @@ public class IgniteImpl implements Ignite {
                 raftMgr,
                 baselineMgr,
                 clusterSvc.topologyService(),
-                getPartitionsStorePath(workDir)
+                getPartitionsStorePath(workDir),
+                txManager
         );
 
         qryEngine = new SqlQueryProcessor(
@@ -287,6 +296,7 @@ public class IgniteImpl implements Ignite {
                     nettyBootstrapFactory,
                     clusterSvc,
                     raftMgr,
+                    txManager,
                     metaStorageMgr,
                     clusterCfgMgr,
                     baselineMgr,
@@ -334,7 +344,7 @@ public class IgniteImpl implements Ignite {
         });
 
         if (explicitStop.get()) {
-            doStopNode(List.of(vaultMgr, nodeCfgMgr, clusterSvc, raftMgr, metaStorageMgr, clusterCfgMgr, baselineMgr,
+            doStopNode(List.of(vaultMgr, nodeCfgMgr, clusterSvc, raftMgr, txManager, metaStorageMgr, clusterCfgMgr, baselineMgr,
                     distributedTblMgr, qryEngine, restModule, clientHandlerModule, nettyBootstrapFactory));
         }
     }
diff --git a/modules/schema/pom.xml b/modules/schema/pom.xml
index fe9b781..40139cf 100644
--- a/modules/schema/pom.xml
+++ b/modules/schema/pom.xml
@@ -50,7 +50,8 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>ignite-metastorage</artifactId>
+            <artifactId>ignite-configuration</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <!-- 3rd party dependencies -->
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
index 5817cc1..cfc5914 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
@@ -169,8 +169,13 @@ public class ByteBufferRow implements BinaryRow {
     }
 
     /** {@inheritDoc} */
-    @Override
-    public byte[] bytes() {
-        return buf.array();
+    @Override public byte[] bytes() {
+        // TODO IGNITE-15934 avoid copy.
+        byte[] tmp = new byte[buf.limit()];
+
+        buf.get(tmp);
+        buf.rewind();
+
+        return tmp;
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/registry/SchemaRegistryImpl.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/registry/SchemaRegistryImpl.java
index c1ba18b..333942a 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/registry/SchemaRegistryImpl.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/registry/SchemaRegistryImpl.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.schema.registry;
 
+import static java.util.stream.Collectors.toList;
+
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -24,7 +26,6 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.function.Function;
 import java.util.function.IntSupplier;
-import java.util.stream.Collectors;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaRegistry;
@@ -148,8 +149,7 @@ public class SchemaRegistryImpl implements SchemaRegistry {
     public Collection<Row> resolve(Collection<BinaryRow> rows) {
         final SchemaDescriptor curSchema = waitLatestSchema();
 
-        return rows.stream().map(row -> resolveInternal(row, curSchema))
-                .collect(Collectors.toList());
+        return rows.stream().map(row -> resolveInternal(row, curSchema)).collect(toList());
     }
 
     /**
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
index 43ed1b2..2eb9d2f 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
@@ -63,14 +63,14 @@ import org.junit.jupiter.api.Test;
 public class SchemaConfigurationConverterTest {
     /** Table builder. */
     private TableDefinitionBuilder tblBuilder;
-    
+
     /** Configuration registry with one table for each test. */
     private ConfigurationRegistry confRegistry;
-    
+
     /**
      * Prepare configuration registry for test.
      *
-     * @throws ExecutionException   If failed.
+     * @throws ExecutionException If failed.
      * @throws InterruptedException If failed.
      */
     @BeforeEach
@@ -82,9 +82,9 @@ public class SchemaConfigurationConverterTest {
                 List.of(),
                 List.of(HashIndexConfigurationSchema.class, SortedIndexConfigurationSchema.class, PartialIndexConfigurationSchema.class)
         );
-        
+
         confRegistry.start();
-        
+
         tblBuilder = SchemaBuilders.tableBuilder("SNAME", "TNAME")
                 .columns(
                         SchemaBuilders.column("COL1", ColumnType.DOUBLE).build(),
@@ -93,9 +93,9 @@ public class SchemaConfigurationConverterTest {
                         SchemaBuilders.column("B", ColumnType.INT8).build(),
                         SchemaBuilders.column("C", ColumnType.INT8).build()
                 ).withPrimaryKey("COL1");
-        
+
         TableDefinition tbl = tblBuilder.build();
-        
+
         confRegistry.getConfiguration(TablesConfiguration.KEY).change(
                 ch -> SchemaConfigurationConverter.createTable(tbl, ch)
                         .changeTables(
@@ -103,12 +103,12 @@ public class SchemaConfigurationConverterTest {
                         )
         ).get();
     }
-    
+
     @AfterEach
     void tearDown() {
         confRegistry.stop();
     }
-    
+
     /**
      * Add/remove HashIndex into configuration and read it back.
      */
@@ -118,36 +118,36 @@ public class SchemaConfigurationConverterTest {
                 .withColumns("A", "B", "C")
                 .withHints(Collections.singletonMap("param", "value"));
         HashIndexDefinition idx = builder.build();
-        
+
         getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
-        
+
         TableDefinition tbl = SchemaConfigurationConverter.convert(getTbl().value());
-        
+
         HashIndexDefinition idx2 = (HashIndexDefinition) getIdx(idx.name(), tbl.indices());
-        
+
         assertNotNull(idx2);
         assertEquals("HASH", idx2.type());
         assertEquals(3, idx2.columns().size());
     }
-    
+
     /**
      * Add/remove SortedIndex into configuration and read it back.
      */
     @Test
     public void testConvertSortedIndex() throws Exception {
         SortedIndexDefinitionBuilder builder = SchemaBuilders.sortedIndex("SIDX");
-        
+
         builder.addIndexColumn("A").asc().done();
         builder.addIndexColumn("B").desc().done();
-        
+
         SortedIndexDefinition idx = builder.build();
-        
+
         getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
-        
+
         TableDefinition tbl = SchemaConfigurationConverter.convert(getTbl().value());
-        
+
         SortedIndexDefinition idx2 = (SortedIndexDefinition) getIdx(idx.name(), tbl.indices());
-        
+
         assertNotNull(idx2);
         assertEquals("SORTED", idx2.type());
         assertEquals(2, idx2.columns().size());
@@ -156,7 +156,7 @@ public class SchemaConfigurationConverterTest {
         assertEquals(SortOrder.ASC, idx2.columns().get(0).sortOrder());
         assertEquals(SortOrder.DESC, idx2.columns().get(1).sortOrder());
     }
-    
+
     /**
      * Add/remove index on primary key into configuration and read it back.
      */
@@ -166,13 +166,13 @@ public class SchemaConfigurationConverterTest {
                 .addIndexColumn("COL1").desc().done()
                 .unique(true)
                 .build();
-        
+
         getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
-        
+
         TableDefinition tbl = SchemaConfigurationConverter.convert(getTbl().value());
-        
+
         SortedIndexDefinition idx2 = (SortedIndexDefinition) getIdx(idx.name(), tbl.indices());
-        
+
         assertNotNull(idx2);
         assertEquals("pk_sorted", idx2.name());
         assertEquals("SORTED", idx2.type());
@@ -180,7 +180,7 @@ public class SchemaConfigurationConverterTest {
                 idx2.columns().stream().map(IndexColumnDefinition::name).collect(Collectors.toList()));
         assertTrue(idx2.unique());
     }
-    
+
     /**
      * Detect an index containing affinity key as unique one.
      */
@@ -191,68 +191,67 @@ public class SchemaConfigurationConverterTest {
                 .addIndexColumn("A").done()
                 .addIndexColumn("COL1").desc().done()
                 .build();
-        
+
         getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
-        
+
         TableDefinition tbl = SchemaConfigurationConverter.convert(getTbl().value());
-        
+
         SortedIndexDefinition idx2 = (SortedIndexDefinition) getIdx(idx.name(), tbl.indices());
-        
+
         assertNotNull(idx2);
         assertEquals("uniq_sorted", idx2.name());
         assertEquals("SORTED", idx2.type());
-        
+
         assertTrue(idx2.unique());
-        
+
         assertEquals(2, idx2.columns().size());
         assertEquals("A", idx2.columns().get(0).name());
         assertEquals("COL1", idx2.columns().get(1).name());
         assertEquals(SortOrder.ASC, idx2.columns().get(0).sortOrder());
         assertEquals(SortOrder.DESC, idx2.columns().get(1).sortOrder());
     }
-    
+
     /**
      * Add/remove PartialIndex into configuration and read it back.
      */
     @Test
     public void testPartialIndex() throws Exception {
         PartialIndexDefinitionBuilder builder = SchemaBuilders.partialIndex("TEST");
-        
+
         builder.addIndexColumn("A").done();
         builder.withExpression("WHERE A > 0");
-        
+
         PartialIndexDefinition idx = builder.build();
-        
+
         getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
-        
+
         TableDefinition tbl = SchemaConfigurationConverter.convert(getTbl().value());
-        
+
         PartialIndexDefinition idx2 = (PartialIndexDefinition) getIdx(idx.name(), tbl.indices());
-        
+
         assertNotNull(idx2);
         assertEquals("PARTIAL", idx2.type());
         assertEquals(idx.columns().size(), idx2.columns().size());
     }
-    
+
     /**
      * Add/remove table and read it back.
      */
     @Test
     public void testConvertTable() {
         TableDefinition tbl = tblBuilder.build();
-        
-        TableConfiguration tblCfg = confRegistry.getConfiguration(TablesConfiguration.KEY).tables()
-                .get(tbl.canonicalName());
-        
+
+        TableConfiguration tblCfg = confRegistry.getConfiguration(TablesConfiguration.KEY).tables().get(tbl.canonicalName());
+
         TableDefinition tbl2 = SchemaConfigurationConverter.convert(tblCfg);
-        
+
         assertEquals(tbl.canonicalName(), tbl2.canonicalName());
         assertEquals(tbl.indices().size(), tbl2.indices().size());
         assertEquals(tbl.keyColumns().size(), tbl2.keyColumns().size());
         assertEquals(tbl.affinityColumns().size(), tbl2.affinityColumns().size());
         assertEquals(tbl.columns().size(), tbl2.columns().size());
     }
-    
+
     /**
      * Get tests default table configuration.
      *
@@ -261,7 +260,7 @@ public class SchemaConfigurationConverterTest {
     private TableConfiguration getTbl() {
         return confRegistry.getConfiguration(TablesConfiguration.KEY).tables().get(tblBuilder.build().canonicalName());
     }
-    
+
     /**
      * Get table index by name.
      *
diff --git a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/ConcurrentHashMapPartitionStorage.java b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/ConcurrentHashMapPartitionStorage.java
index 9f04eec..c16ea3a 100644
--- a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/ConcurrentHashMapPartitionStorage.java
+++ b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/ConcurrentHashMapPartitionStorage.java
@@ -30,10 +30,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.storage.DataRow;
@@ -55,7 +56,7 @@ public class ConcurrentHashMapPartitionStorage implements PartitionStorage {
     private static final String SNAPSHOT_FILE = "snapshot_file";
 
     /** Storage content. */
-    private final ConcurrentMap<ByteArray, byte[]> map = new ConcurrentHashMap<>();
+    private final ConcurrentSkipListMap<ByteArray, byte[]> map = new ConcurrentSkipListMap<>();
 
     /** {@inheritDoc} */
     @Override
@@ -174,10 +175,10 @@ public class ConcurrentHashMapPartitionStorage implements PartitionStorage {
                 map.remove(mapKey);
 
                 break;
-    
+
             case NOOP:
                 break;
-    
+
             default:
                 throw new UnsupportedOperationException(String.valueOf(clo.operationType()));
         }
@@ -262,4 +263,35 @@ public class ConcurrentHashMapPartitionStorage implements PartitionStorage {
     public void close() throws Exception {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        ConcurrentHashMapPartitionStorage that = (ConcurrentHashMapPartitionStorage) o;
+
+        if (!map.equals(that.map)) {
+            return false;
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int hashCode() {
+        int hash = 0;
+
+        for (Map.Entry<ByteArray, byte[]> entry : map.entrySet()) {
+            hash += entry.getKey().hashCode() ^ Arrays.hashCode(entry.getValue());
+        }
+
+        return hash;
+    }
 }
diff --git a/modules/table/pom.xml b/modules/table/pom.xml
index 37acb4e..88a1f16 100644
--- a/modules/table/pom.xml
+++ b/modules/table/pom.xml
@@ -93,6 +93,11 @@
             <artifactId>ignite-storage-rocksdb</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-transactions</artifactId>
+        </dependency>
+
         <!-- Test dependencies -->
         <dependency>
             <groupId>org.apache.ignite</groupId>
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItDistributedTableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItDistributedTableTest.java
deleted file mode 100644
index 0d23602..0000000
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItDistributedTableTest.java
+++ /dev/null
@@ -1,606 +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.ignite.distributed;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import org.apache.ignite.internal.affinity.RendezvousAffinityFunction;
-import org.apache.ignite.internal.raft.Loza;
-import org.apache.ignite.internal.raft.server.RaftServer;
-import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
-import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
-import org.apache.ignite.internal.schema.Column;
-import org.apache.ignite.internal.schema.NativeTypes;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.SchemaRegistry;
-import org.apache.ignite.internal.schema.row.Row;
-import org.apache.ignite.internal.schema.row.RowAssembler;
-import org.apache.ignite.internal.storage.basic.ConcurrentHashMapPartitionStorage;
-import org.apache.ignite.internal.storage.engine.TableStorage;
-import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.internal.table.distributed.command.GetCommand;
-import org.apache.ignite.internal.table.distributed.command.InsertCommand;
-import org.apache.ignite.internal.table.distributed.command.response.SingleRowResponse;
-import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
-import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
-import org.apache.ignite.internal.testframework.WorkDirectory;
-import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
-import org.apache.ignite.internal.thread.NamedThreadFactory;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.lang.IgniteLogger;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.network.ClusterNode;
-import org.apache.ignite.network.ClusterService;
-import org.apache.ignite.network.MessageSerializationRegistryImpl;
-import org.apache.ignite.network.NetworkAddress;
-import org.apache.ignite.network.NodeFinder;
-import org.apache.ignite.network.StaticNodeFinder;
-import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
-import org.apache.ignite.network.serialization.MessageSerializationRegistry;
-import org.apache.ignite.raft.client.Peer;
-import org.apache.ignite.raft.client.service.RaftGroupService;
-import org.apache.ignite.raft.jraft.RaftMessagesFactory;
-import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl;
-import org.apache.ignite.table.KeyValueView;
-import org.apache.ignite.table.RecordView;
-import org.apache.ignite.table.Table;
-import org.apache.ignite.table.Tuple;
-import org.apache.ignite.utils.ClusterServiceTestUtils;
-import org.jetbrains.annotations.NotNull;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.TestInfo;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.mockito.Mockito;
-
-/**
- * Distributed internal table tests.
- */
-@ExtendWith(WorkDirectoryExtension.class)
-public class ItDistributedTableTest {
-    /** The logger. */
-    private static final IgniteLogger LOG = IgniteLogger.forClass(ItDistributedTableTest.class);
-
-    /** Base network port. */
-    public static final int NODE_PORT_BASE = 20_000;
-
-    /** Nodes. */
-    public static final int NODES = 5;
-
-    /** Partitions. */
-    public static final int PARTS = 10;
-
-    /** Factory. */
-    private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
-
-    /** Network factory. */
-    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
-
-    private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
-
-    /** Client. */
-    private ClusterService client;
-
-    /** Executor for raft group services. */
-    ScheduledExecutorService executor;
-
-    /** Schema. */
-    public static SchemaDescriptor SCHEMA = new SchemaDescriptor(
-            1,
-            new Column[]{new Column("key", NativeTypes.INT64, false)},
-            new Column[]{new Column("value", NativeTypes.INT64, false)}
-    );
-
-    /** Cluster. */
-    private ArrayList<ClusterService> cluster = new ArrayList<>();
-
-    @WorkDirectory
-    private Path dataPath;
-
-    /**
-     * Start all cluster nodes before each test.
-     */
-    @BeforeEach
-    public void beforeTest(TestInfo testInfo) {
-        List<NetworkAddress> addresses = ClusterServiceTestUtils.findLocalAddresses(NODE_PORT_BASE, NODE_PORT_BASE + NODES);
-
-        var finder = new StaticNodeFinder(addresses);
-
-        addresses.stream()
-                .map(addr -> startClient(testInfo, addr.port(), finder))
-                .forEach(cluster::add);
-
-        for (ClusterService node : cluster) {
-            assertTrue(waitForTopology(node, NODES, 1000));
-        }
-
-        LOG.info("Cluster started.");
-
-        client = startClient(testInfo, NODE_PORT_BASE + NODES, finder);
-
-        assertTrue(waitForTopology(client, NODES + 1, 1000));
-
-        LOG.info("Client started.");
-
-        executor = new ScheduledThreadPoolExecutor(20, new NamedThreadFactory(Loza.CLIENT_POOL_NAME));
-    }
-
-    /**
-     * Shutdowns all cluster nodes after each test.
-     *
-     * @throws Exception If failed.
-     */
-    @AfterEach
-    public void afterTest() throws Exception {
-        for (ClusterService node : cluster) {
-            node.stop();
-        }
-
-        IgniteUtils.shutdownAndAwaitTermination(executor, 10, TimeUnit.SECONDS);
-
-        client.stop();
-    }
-
-    /**
-     * Tests partition listener.
-     *
-     * @throws Exception If failed.
-     */
-    @Test
-    public void partitionListener() throws Exception {
-        String grpId = "part";
-
-        RaftServer partSrv = new JraftServerImpl(cluster.get(0), dataPath);
-
-        partSrv.start();
-
-        List<Peer> conf = List.of(new Peer(cluster.get(0).topologyService().localMember().address()));
-
-        partSrv.startRaftGroup(
-                grpId,
-                new PartitionListener(new ConcurrentHashMapPartitionStorage()),
-                conf
-        );
-
-        RaftGroupService partRaftGrp =
-                RaftGroupServiceImpl
-                        .start(grpId, client, FACTORY, 10_000, conf, true, 200, executor)
-                        .get(3, TimeUnit.SECONDS);
-
-        Row testRow = getTestRow();
-
-        CompletableFuture<Boolean> insertFur = partRaftGrp.run(new InsertCommand(testRow));
-
-        assertTrue(insertFur.get());
-
-        Row keyChunk = getTestKey();
-
-        CompletableFuture<SingleRowResponse> getFut = partRaftGrp.run(new GetCommand(keyChunk));
-
-        assertNotNull(getFut.get().getValue());
-
-        assertEquals(testRow.longValue(1), new Row(SCHEMA, getFut.get().getValue()).longValue(1));
-
-        partSrv.stopRaftGroup(grpId);
-
-        partRaftGrp.shutdown();
-
-        partSrv.stop();
-    }
-
-    /**
-     * Prepares a test row which contains one field.
-     *
-     * @return Row.
-     */
-    @NotNull
-    private Row getTestKey() {
-        RowAssembler rowBuilder = new RowAssembler(SCHEMA, 0, 0);
-
-        rowBuilder.appendLong(1L);
-
-        return new Row(SCHEMA, new ByteBufferRow(rowBuilder.toBytes()));
-    }
-
-    /**
-     * Prepares a test row which contains two fields.
-     *
-     * @return Row.
-     */
-    @NotNull
-    private Row getTestRow() {
-        RowAssembler rowBuilder = new RowAssembler(SCHEMA, 0, 0);
-
-        rowBuilder.appendLong(1L);
-        rowBuilder.appendLong(10L);
-
-        return new Row(SCHEMA, new ByteBufferRow(rowBuilder.toBytes()));
-    }
-
-    /**
-     * The test prepares a distributed table and checks operation over various views.
-     *
-     * @throws Exception If failed.
-     */
-    @Test
-    public void partitionedTable() throws Exception {
-        HashMap<ClusterNode, RaftServer> raftServers = new HashMap<>(NODES);
-
-        for (int i = 0; i < NODES; i++) {
-            var raftSrv = new JraftServerImpl(cluster.get(i), dataPath);
-
-            raftSrv.start();
-
-            raftServers.put(cluster.get(i).topologyService().localMember(), raftSrv);
-        }
-
-        List<List<ClusterNode>> assignment = RendezvousAffinityFunction.assignPartitions(
-                cluster.stream().map(node -> node.topologyService().localMember()).collect(Collectors.toList()),
-                PARTS,
-                1,
-                false,
-                null
-        );
-
-        int p = 0;
-
-        Map<Integer, RaftGroupService> partMap = new HashMap<>();
-
-        for (List<ClusterNode> partNodes : assignment) {
-            String grpId = "part-" + p;
-
-            List<Peer> conf = List.of(new Peer(partNodes.get(0).address()));
-
-            for (ClusterNode node : partNodes) {
-                RaftServer rs = raftServers.get(node);
-
-                rs.startRaftGroup(
-                        grpId,
-                        new PartitionListener(new ConcurrentHashMapPartitionStorage()),
-                        conf
-                );
-            }
-
-            RaftGroupService service = RaftGroupServiceImpl.start(grpId,
-                    client,
-                    FACTORY,
-                    10_000,
-                    conf,
-                    true,
-                    200,
-                    executor
-            ).get(3, TimeUnit.SECONDS);
-
-            partMap.put(p, service);
-
-            p++;
-        }
-
-        Table tbl = new TableImpl(new InternalTableImpl(
-                "tbl",
-                new IgniteUuid(UUID.randomUUID(), 0),
-                partMap,
-                PARTS,
-                NetworkAddress::toString,
-                Mockito.mock(TableStorage.class)
-        ), new SchemaRegistry() {
-            @Override
-            public SchemaDescriptor schema() {
-                return SCHEMA;
-            }
-    
-            @Override
-            public SchemaDescriptor schema(int ver) {
-                return SCHEMA;
-            }
-            
-            @Override
-            public SchemaDescriptor waitLatestSchema() {
-                return schema();
-            }
-    
-            @Override
-            public int lastSchemaVersion() {
-                return SCHEMA.version();
-            }
-
-            @Override
-            public Row resolve(BinaryRow row) {
-                return new Row(SCHEMA, row);
-            }
-
-            @Override public Collection<Row> resolve(Collection<BinaryRow> rows) {
-                return rows.stream().map(this::resolve).collect(Collectors.toList());
-            }
-        }, null);
-
-        partitionedTableRecordView(tbl.recordView(), PARTS * 10);
-
-        partitionedTableKeyValueView(tbl.keyValueView(), PARTS * 10);
-
-        p = 0;
-
-        for (List<ClusterNode> partNodes : assignment) {
-            String grpId = "part-" + p;
-
-            for (ClusterNode node : partNodes) {
-                raftServers.get(node).stopRaftGroup(grpId);
-            }
-
-            p++;
-        }
-
-        for (RaftGroupService srvc : partMap.values()) {
-            srvc.shutdown();
-        }
-
-        for (RaftServer rs : raftServers.values()) {
-            rs.stop();
-        }
-    }
-
-    /**
-     * Checks operation over row table view.
-     *
-     * @param view    Table view.
-     * @param keysCnt Count of keys.
-     */
-    public void partitionedTableRecordView(RecordView<Tuple> view, int keysCnt) {
-        LOG.info("Test for Table view [keys={}]", keysCnt);
-
-        for (int i = 0; i < keysCnt; i++) {
-            view.insert(Tuple.create()
-                    .set("key", Long.valueOf(i))
-                    .set("value", Long.valueOf(i + 2))
-            );
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 2), entry.longValue("value"));
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            view.upsert(Tuple.create()
-                    .set("key", Long.valueOf(i))
-                    .set("value", Long.valueOf(i + 5))
-            );
-
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 5), entry.longValue("value"));
-        }
-
-        HashSet<Tuple> keys = new HashSet<>();
-
-        for (int i = 0; i < keysCnt; i++) {
-            keys.add(Tuple.create().set("key", Long.valueOf(i)));
-        }
-
-        Collection<Tuple> entries = view.getAll(keys);
-
-        assertEquals(keysCnt, entries.size());
-
-        for (int i = 0; i < keysCnt; i++) {
-            boolean res = view.replace(
-                    Tuple.create()
-                            .set("key", Long.valueOf(i))
-                            .set("value", Long.valueOf(i + 5)),
-                    Tuple.create()
-                            .set("key", Long.valueOf(i))
-                            .set("value", Long.valueOf(i + 2))
-            );
-
-            assertTrue(res);
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            boolean res = view.delete(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertTrue(res);
-
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertNull(entry);
-        }
-
-        ArrayList<Tuple> batch = new ArrayList<>(keysCnt);
-
-        for (int i = 0; i < keysCnt; i++) {
-            batch.add(Tuple.create()
-                    .set("key", Long.valueOf(i))
-                    .set("value", Long.valueOf(i + 2))
-            );
-        }
-
-        view.upsertAll(batch);
-
-        for (int i = 0; i < keysCnt; i++) {
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 2), entry.longValue("value"));
-        }
-
-        view.deleteAll(keys);
-
-        for (Tuple key : keys) {
-            Tuple entry = view.get(key);
-
-            assertNull(entry);
-        }
-    }
-
-    /**
-     * Checks operation over key-value binary table view.
-     *
-     * @param view    Table view.
-     * @param keysCnt Count of keys.
-     */
-    public void partitionedTableKeyValueView(KeyValueView<Tuple, Tuple> view, int keysCnt) {
-        LOG.info("Tes for Key-Value binary view [keys={}]", keysCnt);
-
-        for (int i = 0; i < keysCnt; i++) {
-            view.putIfAbsent(
-                    Tuple.create().set("key", Long.valueOf(i)),
-                    Tuple.create().set("value", Long.valueOf(i + 2))
-            );
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 2), entry.longValue("value"));
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            view.put(
-                    Tuple.create().set("key", Long.valueOf(i)),
-                    Tuple.create().set("value", Long.valueOf(i + 5))
-            );
-
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 5), entry.longValue("value"));
-        }
-
-        HashSet<Tuple> keys = new HashSet<>();
-
-        for (int i = 0; i < keysCnt; i++) {
-            keys.add(Tuple.create().set("key", Long.valueOf(i)));
-        }
-
-        Map<Tuple, Tuple> entries = view.getAll(keys);
-
-        assertEquals(keysCnt, entries.size());
-
-        for (int i = 0; i < keysCnt; i++) {
-            boolean res = view.replace(
-                    Tuple.create().set("key", Long.valueOf(i)),
-                    Tuple.create().set("value", Long.valueOf(i + 5)),
-                    Tuple.create().set("value", Long.valueOf(i + 2))
-            );
-
-            assertTrue(res);
-        }
-
-        for (int i = 0; i < keysCnt; i++) {
-            boolean res = view.remove(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertTrue(res);
-
-            Tuple entry = view.get(
-                    Tuple.create()
-                            .set("key", Long.valueOf(i))
-            );
-
-            assertNull(entry);
-        }
-
-        HashMap<Tuple, Tuple> batch = new HashMap<>(keysCnt);
-
-        for (int i = 0; i < keysCnt; i++) {
-            batch.put(
-                    Tuple.create().set("key", Long.valueOf(i)),
-                    Tuple.create().set("value", Long.valueOf(i + 2))
-            );
-        }
-
-        view.putAll(batch);
-
-        for (int i = 0; i < keysCnt; i++) {
-            Tuple entry = view.get(Tuple.create().set("key", Long.valueOf(i)));
-
-            assertEquals(Long.valueOf(i + 2), entry.longValue("value"));
-        }
-
-        view.removeAll(keys);
-
-        for (Tuple key : keys) {
-            Tuple entry = view.get(key);
-
-            assertNull(entry);
-        }
-    }
-
-    /**
-     * Starts client.
-     *
-     * @param testInfo   Test info.
-     * @param port       Local port.
-     * @param nodeFinder Node finder.
-     * @return The client cluster view.
-     */
-    private static ClusterService startClient(TestInfo testInfo, int port, NodeFinder nodeFinder) {
-        var network = ClusterServiceTestUtils.clusterService(
-                testInfo,
-                port,
-                nodeFinder,
-                SERIALIZATION_REGISTRY,
-                NETWORK_FACTORY
-        );
-
-        network.start();
-
-        return network;
-    }
-
-    /**
-     * Wait for topology.
-     *
-     * @param cluster  The cluster.
-     * @param expected Expected count.
-     * @param timeout  The timeout in millis.
-     * @return {@code True} if topology size is equal to expected.
-     */
-    private boolean waitForTopology(ClusterService cluster, int expected, int timeout) {
-        long stop = System.currentTimeMillis() + timeout;
-
-        while (System.currentTimeMillis() < stop) {
-            if (cluster.topologyService().allMembers().size() >= expected) {
-                return true;
-            }
-
-            try {
-                Thread.sleep(50);
-            } catch (InterruptedException e) {
-                return false;
-            }
-        }
-
-        return false;
-    }
-}
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
index 9fb7fe2..cf0d41e 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItInternalTableScanTest.java
@@ -50,6 +50,7 @@ import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.internal.raft.server.impl.RaftServerImpl;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
 import org.apache.ignite.internal.storage.DataRow;
 import org.apache.ignite.internal.storage.PartitionStorage;
 import org.apache.ignite.internal.storage.StorageException;
@@ -58,11 +59,17 @@ import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
 import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
+import org.apache.ignite.internal.table.distributed.storage.VersionedRowStore;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.impl.HeapLockManager;
+import org.apache.ignite.internal.tx.impl.TxManagerImpl;
 import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.lang.IgniteUuidGenerator;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
@@ -75,6 +82,7 @@ import org.apache.ignite.raft.jraft.RaftMessagesFactory;
 import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl;
 import org.apache.ignite.utils.ClusterServiceTestUtils;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -84,7 +92,7 @@ import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
- * Tests for {@link InternalTable#scan(int, org.apache.ignite.tx.Transaction)}.
+ * Tests for {@link InternalTable#scan(int, org.apache.ignite.internal.tx.InternalTransaction)}.
  */
 @ExtendWith(MockitoExtension.class)
 public class ItInternalTableScanTest {
@@ -107,6 +115,8 @@ public class ItInternalTableScanTest {
 
     private RaftServer raftSrv;
 
+    private TxManager txManager;
+
     /** Internal table to test. */
     private InternalTable internalTbl;
 
@@ -143,13 +153,26 @@ public class ItInternalTableScanTest {
 
         raftSrv.start();
 
+        String grpName = "test_part_grp";
+
         List<Peer> conf = List.of(new Peer(nodeNetworkAddress));
 
         mockStorage = mock(PartitionStorage.class);
 
+        txManager = new TxManagerImpl(network, new HeapLockManager());
+
+        txManager.start();
+
+        IgniteUuid tblId = new IgniteUuid(UUID.randomUUID(), 0);
+
         raftSrv.startRaftGroup(
-                RAFT_GRP_ID,
-                new PartitionListener(mockStorage),
+                grpName,
+                new PartitionListener(tblId, new VersionedRowStore(mockStorage, txManager) {
+                    @Override
+                    protected Pair<BinaryRow, BinaryRow> versionedRow(@Nullable DataRow row, Timestamp timestamp) {
+                        return new Pair<>(new ByteBufferRow(row.valueBytes()), null); // Return as is.
+                    }
+                }),
                 conf
         );
 
@@ -168,10 +191,11 @@ public class ItInternalTableScanTest {
 
         internalTbl = new InternalTableImpl(
                 TEST_TABLE_NAME,
-                new IgniteUuidGenerator(UUID.randomUUID(), 0).randomUuid(),
+                tblId,
                 Map.of(0, raftGrpSvc),
                 1,
                 NetworkAddress::toString,
+                txManager,
                 mock(TableStorage.class)
         );
     }
@@ -202,6 +226,10 @@ public class ItInternalTableScanTest {
         if (network != null) {
             network.stop();
         }
+
+        if (txManager != null) {
+            txManager.stop();
+        }
     }
 
     /**
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
index 6ff1dd0..330de6c 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
@@ -17,34 +17,41 @@
 
 package org.apache.ignite.distributed;
 
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.mockito.Mockito.mock;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BooleanSupplier;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
+import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.NativeTypes;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.schema.row.RowAssembler;
-import org.apache.ignite.internal.storage.DataRow;
-import org.apache.ignite.internal.storage.PartitionStorage;
 import org.apache.ignite.internal.storage.basic.ConcurrentHashMapPartitionStorage;
-import org.apache.ignite.internal.storage.basic.SimpleDataRow;
 import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.table.distributed.TableTxManagerImpl;
 import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
 import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
+import org.apache.ignite.internal.table.distributed.storage.VersionedRowStore;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.impl.HeapLockManager;
+import org.apache.ignite.internal.tx.impl.TxManagerImpl;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.raft.client.service.ItAbstractListenerSnapshotTest;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
 import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.junit.jupiter.api.AfterEach;
 
 /**
  * Persistent partitions raft group snapshots tests.
@@ -64,18 +71,39 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
-    /** Paths for created partition listeners. */
+    /**
+     * Paths for created partition listeners.
+     */
     private final Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();
 
+    private final List<TxManager> managers = new ArrayList<>();
+
+    @AfterEach
+    @Override
+    public void afterTest() throws Exception {
+        super.afterTest();
+
+        for (TxManager txManager : managers) {
+            txManager.stop();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override
     public void beforeFollowerStop(RaftGroupService service) throws Exception {
+        TxManagerImpl txManager = new TxManagerImpl(clientService(), new HeapLockManager());
+
+        managers.add(txManager);
+
+        txManager.start();
+
         var table = new InternalTableImpl(
                 "table",
                 new IgniteUuid(UUID.randomUUID(), 0),
                 Map.of(0, service),
                 1,
                 NetworkAddress::toString,
+                txManager,
                 mock(TableStorage.class)
         );
 
@@ -85,12 +113,19 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
     /** {@inheritDoc} */
     @Override
     public void afterFollowerStop(RaftGroupService service) throws Exception {
+        TxManagerImpl txManager = new TxManagerImpl(clientService(), new HeapLockManager());
+
+        managers.add(txManager);
+
+        txManager.start();
+
         var table = new InternalTableImpl(
                 "table",
                 new IgniteUuid(UUID.randomUUID(), 0),
                 Map.of(0, service),
                 1,
                 NetworkAddress::toString,
+                txManager,
                 mock(TableStorage.class)
         );
 
@@ -99,41 +134,52 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
         // Put deleted data again
         table.upsert(FIRST_VALUE, null).get();
+
+        txManager.stop();
     }
 
     /** {@inheritDoc} */
     @Override
     public void afterSnapshot(RaftGroupService service) throws Exception {
+        TxManager txManager = new TxManagerImpl(clientService(), new HeapLockManager());
+
+        managers.add(txManager);
+
+        txManager.start();
+
         var table = new InternalTableImpl(
                 "table",
                 new IgniteUuid(UUID.randomUUID(), 0),
                 Map.of(0, service),
                 1,
                 NetworkAddress::toString,
+                txManager,
                 mock(TableStorage.class)
         );
 
         table.upsert(SECOND_VALUE, null).get();
+
+        assertNotNull(table.get(SECOND_KEY, null).join());
+
+        txManager.stop();
     }
 
     /** {@inheritDoc} */
     @Override
     public BooleanSupplier snapshotCheckClosure(JraftServerImpl restarted, boolean interactedAfterSnapshot) {
-        PartitionStorage storage = getListener(restarted, raftGroupId()).getStorage();
+        VersionedRowStore storage = getListener(restarted, raftGroupId()).getStorage();
 
         Row key = interactedAfterSnapshot ? SECOND_KEY : FIRST_KEY;
         Row value = interactedAfterSnapshot ? SECOND_VALUE : FIRST_VALUE;
 
-        ByteBuffer buffer = key.keySlice();
-        byte[] keyBytes = new byte[buffer.capacity()];
-        buffer.get(keyBytes);
+        return () -> {
+            BinaryRow read = storage.get(key, null);
 
-        SimpleDataRow finalRow = new SimpleDataRow(keyBytes, null);
-        SimpleDataRow finalValue = new SimpleDataRow(keyBytes, value.bytes());
+            if (read == null) {
+                return false;
+            }
 
-        return () -> {
-            DataRow read = storage.read(finalRow);
-            return Objects.equals(finalValue, read);
+            return Arrays.equals(value.bytes(), read.bytes());
         };
     }
 
@@ -145,13 +191,19 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
     /** {@inheritDoc} */
     @Override
-    public RaftGroupListener createListener(Path workDir) {
+    public RaftGroupListener createListener(ClusterService service, Path workDir) {
         return paths.entrySet().stream()
                 .filter(entry -> entry.getValue().equals(workDir))
                 .map(Map.Entry::getKey)
                 .findAny()
                 .orElseGet(() -> {
-                    PartitionListener listener = new PartitionListener(new ConcurrentHashMapPartitionStorage());
+                    TableTxManagerImpl txManager = new TableTxManagerImpl(service, new HeapLockManager());
+
+                    txManager.start(); // Init listener.
+
+                    PartitionListener listener = new PartitionListener(
+                            new IgniteUuid(UUID.randomUUID(), 0),
+                            new VersionedRowStore(new ConcurrentHashMapPartitionStorage(), txManager));
 
                     paths.put(listener, workDir);
 
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
new file mode 100644
index 0000000..b6bd366
--- /dev/null
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
@@ -0,0 +1,437 @@
+/*
+ * 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.distributed;
+
+import static org.apache.ignite.raft.jraft.test.TestUtils.waitForTopology;
+import static org.apache.ignite.utils.ClusterServiceTestUtils.findLocalAddresses;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.affinity.RendezvousAffinityFunction;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
+import org.apache.ignite.internal.storage.basic.ConcurrentHashMapPartitionStorage;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.internal.table.TxAbstractTest;
+import org.apache.ignite.internal.table.distributed.TableTxManagerImpl;
+import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
+import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
+import org.apache.ignite.internal.table.distributed.storage.VersionedRowStore;
+import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.impl.HeapLockManager;
+import org.apache.ignite.internal.tx.impl.IgniteTransactionsImpl;
+import org.apache.ignite.internal.tx.impl.TxManagerImpl;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.lang.IgniteUuidGenerator;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.MessageSerializationRegistryImpl;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NodeFinder;
+import org.apache.ignite.network.StaticNodeFinder;
+import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
+import org.apache.ignite.network.serialization.MessageSerializationRegistry;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.apache.ignite.raft.jraft.RaftMessagesFactory;
+import org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.utils.ClusterServiceTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.mockito.Mockito;
+
+/**
+ * Distributed transaction test using a single partition table.
+ */
+public class ItTxDistributedTestSingleNode extends TxAbstractTest {
+    public static final int NODE_PORT_BASE = 20_000;
+
+    private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
+
+    private static final TestScaleCubeClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+
+    private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistryImpl();
+
+    private ClusterService client;
+
+    protected Map<ClusterNode, Loza> raftServers;
+
+    protected Map<ClusterNode, TxManager> txManagers;
+
+    protected Map<Integer, RaftGroupService> accRaftClients;
+
+    protected Map<Integer, RaftGroupService> custRaftClients;
+
+    protected List<ClusterService> cluster = new CopyOnWriteArrayList<>();
+
+    private ScheduledThreadPoolExecutor executor;
+
+    /**
+     * Returns a count of nodes.
+     *
+     * @return Nodes.
+     */
+    protected int nodes() {
+        return 1;
+    }
+
+    /**
+     * Returns a count of replicas.
+     *
+     * @return Replicas.
+     */
+    protected int replicas() {
+        return 1;
+    }
+
+    /**
+     * Returns {@code true} to disable collocation by using dedicated client node.
+     *
+     * @return {@code true} to disable collocation.
+     */
+    protected boolean startClient() {
+        return true;
+    }
+
+    private final TestInfo testInfo;
+
+    /**
+     * The constructor.
+     *
+     * @param testInfo Test info.
+     */
+    public ItTxDistributedTestSingleNode(TestInfo testInfo) {
+        this.testInfo = testInfo;
+    }
+
+    /**
+     * Initialize the test state.
+     */
+    @Override
+    @BeforeEach
+    public void before() throws Exception {
+        int nodes = nodes();
+        int replicas = replicas();
+
+        assertTrue(nodes > 0);
+        assertTrue(replicas > 0);
+
+        List<NetworkAddress> localAddresses = findLocalAddresses(NODE_PORT_BASE,
+                NODE_PORT_BASE + nodes);
+
+        var nodeFinder = new StaticNodeFinder(localAddresses);
+
+        nodeFinder.findNodes().parallelStream()
+                .map(addr -> startNode(testInfo, addr.toString(), addr.port(), nodeFinder))
+                .forEach(cluster::add);
+
+        for (ClusterService node : cluster) {
+            assertTrue(waitForTopology(node, nodes, 1000));
+        }
+
+        log.info("The cluster has been started");
+
+        if (startClient()) {
+            client = startNode(testInfo, "client", NODE_PORT_BASE - 1, nodeFinder);
+
+            assertTrue(waitForTopology(client, nodes + 1, 1000));
+
+            log.info("The client has been started");
+        }
+
+        // Start raft servers. Each raft server can hold multiple groups.
+        raftServers = new HashMap<>(nodes);
+        txManagers = new HashMap<>(nodes);
+
+        executor = new ScheduledThreadPoolExecutor(20,
+                new NamedThreadFactory(Loza.CLIENT_POOL_NAME));
+
+        for (int i = 0; i < nodes; i++) {
+            var raftSrv = new Loza(cluster.get(i), workDir);
+
+            raftSrv.start();
+
+            ClusterNode node = cluster.get(i).topologyService().localMember();
+
+            raftServers.put(node, raftSrv);
+
+            TableTxManagerImpl txMgr = new TableTxManagerImpl(cluster.get(i), new HeapLockManager());
+
+            txMgr.start();
+
+            txManagers.put(node, txMgr);
+        }
+
+        log.info("Raft servers have been started");
+
+        final String accountsName = "accounts";
+        final String customersName = "customers";
+
+        IgniteUuid accTblId = new IgniteUuidGenerator(UUID.randomUUID(), 0).randomUuid();
+        IgniteUuid custTblId = new IgniteUuidGenerator(UUID.randomUUID(), 0).randomUuid();
+
+        accRaftClients = startTable(accountsName, accTblId);
+        custRaftClients = startTable(customersName, custTblId);
+
+        log.info("Partition groups have been started");
+
+        TxManager txMgr;
+
+        if (startClient()) {
+            txMgr = new TxManagerImpl(client, new HeapLockManager());
+        } else {
+            // Collocated mode.
+            txMgr = txManagers.get(accRaftClients.get(0).clusterService().topologyService().localMember());
+        }
+
+        assertNotNull(txMgr);
+
+        igniteTransactions = new IgniteTransactionsImpl(txMgr);
+
+        this.accounts = new TableImpl(new InternalTableImpl(
+                accountsName,
+                accTblId,
+                accRaftClients,
+                1,
+                NetworkAddress::toString,
+                txMgr,
+                Mockito.mock(TableStorage.class)
+        ), new DummySchemaManagerImpl(ACCOUNTS_SCHEMA), null);
+
+        this.customers = new TableImpl(new InternalTableImpl(
+                customersName,
+                custTblId,
+                custRaftClients,
+                1,
+                NetworkAddress::toString,
+                txMgr,
+                Mockito.mock(TableStorage.class)
+        ), new DummySchemaManagerImpl(CUSTOMERS_SCHEMA), null);
+
+        log.info("Tables have been started");
+    }
+
+    /**
+     * Starts a table.
+     *
+     * @param name The name.
+     * @param tblId Table id.
+     * @return Groups map.
+     */
+    protected Map<Integer, RaftGroupService> startTable(String name, IgniteUuid tblId)
+            throws Exception {
+        List<List<ClusterNode>> assignment = RendezvousAffinityFunction.assignPartitions(
+                cluster.stream().map(node -> node.topologyService().localMember())
+                        .collect(Collectors.toList()),
+                1,
+                replicas(),
+                false,
+                null
+        );
+
+        Map<Integer, RaftGroupService> clients = new HashMap<>();
+
+        for (int p = 0; p < assignment.size(); p++) {
+            List<ClusterNode> partNodes = assignment.get(p);
+
+            String grpId = name + "-part-" + p;
+
+            List<Peer> conf = partNodes.stream().map(n -> n.address()).map(Peer::new)
+                    .collect(Collectors.toList());
+
+            for (ClusterNode node : partNodes) {
+                raftServers.get(node).prepareRaftGroup(
+                        grpId,
+                        partNodes,
+                        () -> new PartitionListener(tblId,
+                                new VersionedRowStore(new ConcurrentHashMapPartitionStorage(), txManagers.get(node)))
+                );
+            }
+
+            if (startClient()) {
+                RaftGroupService service = RaftGroupServiceImpl
+                        .start(grpId, client, FACTORY, 10_000, conf, true, 200, executor)
+                        .get(5, TimeUnit.SECONDS);
+
+                clients.put(p, service);
+            } else {
+                // Create temporary client to find a leader address.
+                ClusterService tmpSvc = raftServers.values().stream().findFirst().get().service();
+
+                RaftGroupService service = RaftGroupServiceImpl
+                        .start(grpId, tmpSvc, FACTORY, 10_000, conf, true, 200, executor)
+                        .get(5, TimeUnit.SECONDS);
+
+                Peer leader = service.leader();
+
+                service.shutdown();
+
+                Loza leaderSrv = raftServers
+                        .get(tmpSvc.topologyService().getByAddress(leader.address()));
+
+                RaftGroupService leaderClusterSvc = RaftGroupServiceImpl
+                        .start(grpId, leaderSrv.service(), FACTORY,
+                                10_000, conf, true, 200, executor).get(5, TimeUnit.SECONDS);
+
+                clients.put(p, leaderClusterSvc);
+            }
+        }
+
+        return clients;
+    }
+
+    /**
+     * Returns a raft manager for a group.
+     *
+     * @param svc The service.
+     * @return Raft manager hosting a leader for group.
+     */
+    protected Loza getLeader(RaftGroupService svc) {
+        Peer leader = svc.leader();
+
+        assertNotNull(leader);
+
+        return raftServers.get(svc.clusterService().topologyService().getByAddress(leader.address()));
+    }
+
+    /**
+     * Shutdowns all cluster nodes after each test.
+     *
+     * @throws Exception If failed.
+     */
+    @AfterEach
+    public void after() throws Exception {
+        cluster.parallelStream().map(c -> {
+            c.stop();
+            return null;
+        }).forEach(o -> {
+        });
+
+        if (client != null) {
+            client.stop();
+        }
+
+        IgniteUtils.shutdownAndAwaitTermination(executor, 10, TimeUnit.SECONDS);
+
+        for (Loza rs : raftServers.values()) {
+            Set<String> grps = rs.startedGroups();
+
+            for (String grp : grps) {
+                rs.stopRaftGroup(grp);
+            }
+
+            rs.stop();
+        }
+
+        for (TxManager txMgr : txManagers.values()) {
+            txMgr.stop();
+        }
+
+        for (RaftGroupService svc : accRaftClients.values()) {
+            svc.shutdown();
+        }
+
+        for (RaftGroupService svc : custRaftClients.values()) {
+            svc.shutdown();
+        }
+    }
+
+    /**
+     * Starts a node.
+     *
+     * @param name Node name.
+     * @param port Local port.
+     * @param nodeFinder Node finder.
+     * @return The client cluster view.
+     */
+    protected static ClusterService startNode(TestInfo testInfo, String name, int port,
+            NodeFinder nodeFinder) {
+        var network = ClusterServiceTestUtils.clusterService(
+                testInfo,
+                port,
+                nodeFinder,
+                SERIALIZATION_REGISTRY,
+                NETWORK_FACTORY
+        );
+
+        network.start();
+
+        return network;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected TxManager txManager(Table t) {
+        Map<Integer, RaftGroupService> clients = null;
+
+        if (t == accounts) {
+            clients = accRaftClients;
+        } else if (t == customers) {
+            clients = custRaftClients;
+        } else {
+            fail("Unknown table " + t.name());
+        }
+
+        TxManager manager = txManagers
+                .get(clients.get(0).clusterService().topologyService().getByAddress(clients.get(0).leader().address()));
+
+        assertNotNull(manager);
+
+        return manager;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected boolean assertPartitionsSame(Table table, int partId) {
+        int hash = 0;
+
+        for (Map.Entry<ClusterNode, Loza> entry : raftServers.entrySet()) {
+            Loza svc = (Loza) entry.getValue();
+            JraftServerImpl server = (JraftServerImpl) svc.server();
+            org.apache.ignite.raft.jraft.RaftGroupService grp = server.raftGroupService(table.name() + "-part-" + partId);
+            JraftServerImpl.DelegatingStateMachine fsm = (JraftServerImpl.DelegatingStateMachine) grp
+                    .getRaftNode().getOptions().getFsm();
+            PartitionListener listener = (PartitionListener) fsm.getListener();
+            VersionedRowStore storage = listener.getStorage();
+
+            if (hash == 0) {
+                hash = storage.delegate().hashCode();
+            } else if (hash != storage.delegate().hashCode()) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+}
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeCollocated.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeCollocated.java
new file mode 100644
index 0000000..e8f83e4
--- /dev/null
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNodeCollocated.java
@@ -0,0 +1,56 @@
+/*
+ * 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.distributed;
+
+import static org.junit.jupiter.api.Assertions.assertSame;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * Distributed transaction test using a single partition table, collocated on a leader.
+ */
+public class ItTxDistributedTestSingleNodeCollocated extends ItTxDistributedTestSingleNode {
+    /**
+     * The constructor.
+     *
+     * @param testInfo Test info.
+     */
+    public ItTxDistributedTestSingleNodeCollocated(TestInfo testInfo) {
+        super(testInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected boolean startClient() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @BeforeEach
+    @Override
+    public void before() throws Exception {
+        super.before();
+
+        assertSame(accRaftClients.get(0).clusterService(), getLeader(accRaftClients.get(0)).service());
+        assertSame(custRaftClients.get(0).clusterService(), getLeader(custRaftClients.get(0)).service());
+    }
+}
+
+
+
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
new file mode 100644
index 0000000..d9fb14a
--- /dev/null
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
@@ -0,0 +1,55 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * Distributed transaction test using a single partition table, 3 nodes and 3 replicas.
+ */
+public class ItTxDistributedTestThreeNodesThreeReplicas extends ItTxDistributedTestSingleNode {
+    /**
+     * The constructor.
+     *
+     * @param testInfo Test info.
+     */
+    public ItTxDistributedTestThreeNodesThreeReplicas(TestInfo testInfo) {
+        super(testInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int nodes() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int replicas() {
+        return 3;
+    }
+
+    @Override
+    @AfterEach
+    public void after() throws Exception {
+        IgniteTestUtils.waitForCondition(() -> assertPartitionsSame(accounts, 0), 5_000);
+        IgniteTestUtils.waitForCondition(() -> assertPartitionsSame(customers, 0), 5_000);
+
+        super.after();
+    }
+}
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
new file mode 100644
index 0000000..50062d7
--- /dev/null
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
@@ -0,0 +1,50 @@
+/*
+ * 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.distributed;
+
+import static org.junit.jupiter.api.Assertions.assertSame;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+
+/**
+ * Distributed transaction test using a single partition table, 3 nodes and 3 replicas, collocated on a leader.
+ */
+public class ItTxDistributedTestThreeNodesThreeReplicasCollocated extends ItTxDistributedTestThreeNodesThreeReplicas {
+    /**
+     * The constructor.
+     *
+     * @param testInfo Test info.
+     */
+    public ItTxDistributedTestThreeNodesThreeReplicasCollocated(TestInfo testInfo) {
+        super(testInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean startClient() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @BeforeEach
+    @Override public void before() throws Exception {
+        super.before();
+
+        assertSame(accRaftClients.get(0).clusterService(), getLeader(accRaftClients.get(0)).service());
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/AbstractTableView.java b/modules/table/src/main/java/org/apache/ignite/internal/table/AbstractTableView.java
index 14e5498..03038c0 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/AbstractTableView.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/AbstractTableView.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.table;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.tx.InternalTransaction;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.tx.Transaction;
@@ -31,13 +32,13 @@ import org.jetbrains.annotations.Nullable;
 abstract class AbstractTableView {
     /** Internal table. */
     protected final InternalTable tbl;
-    
+
     /** Schema registry. */
     protected final SchemaRegistry schemaReg;
-    
+
     /** The transaction. */
-    protected final @Nullable Transaction tx;
-    
+    protected final @Nullable InternalTransaction tx;
+
     /**
      * Constructor.
      *
@@ -48,9 +49,9 @@ abstract class AbstractTableView {
     protected AbstractTableView(InternalTable tbl, SchemaRegistry schemaReg, @Nullable Transaction tx) {
         this.tbl = tbl;
         this.schemaReg = schemaReg;
-        this.tx = tx;
+        this.tx = (InternalTransaction) tx;
     }
-    
+
     /**
      * Waits for operation completion.
      *
@@ -63,7 +64,7 @@ abstract class AbstractTableView {
             return fut.get();
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt(); // Restore interrupt flag.
-            
+
             throw convertException(e);
         } catch (ExecutionException e) {
             throw convertException(e.getCause());
@@ -71,14 +72,23 @@ abstract class AbstractTableView {
             throw convertException(e);
         }
     }
-    
+
     /**
      * Returns current transaction.
      */
     public @Nullable Transaction transaction() {
         return tx;
     }
-    
+
+    /**
+     * Enlists a table into a transaction.
+     *
+     * @param tx The transaction.
+     * @return Transactional view.
+     * @deprecated TODO IGNITE-15930 remove and replace with expicit TX argument in table API calls.
+     */
+    public abstract AbstractTableView withTransaction(Transaction tx);
+
     /**
      * Converts an internal exception to a public one.
      *
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
index fea9ba3..85dba7e 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
@@ -23,8 +23,9 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.LockException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -60,8 +61,9 @@ public interface InternalTable extends AutoCloseable {
      * @param keyRow Row with key columns set.
      * @param tx     The transaction.
      * @return Future representing pending completion of the operation.
+     * @throws LockException If a lock can't be acquired by some reason.
      */
-    CompletableFuture<BinaryRow> get(BinaryRow keyRow, @Nullable Transaction tx);
+    CompletableFuture<BinaryRow> get(BinaryRow keyRow, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously get rows from the table.
@@ -70,7 +72,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx      The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows, @Nullable Transaction tx);
+    CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously inserts a row into the table if does not exist or replaces the existed one.
@@ -79,7 +81,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Void> upsert(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<Void> upsert(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously inserts a row into the table if does not exist or replaces the existed one.
@@ -88,7 +90,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx   The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, @Nullable Transaction tx);
+    CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously inserts a row into the table or replaces if exists and return replaced previous row.
@@ -97,7 +99,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously inserts a row into the table if not exists.
@@ -106,7 +108,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Boolean> insert(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<Boolean> insert(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously insert rows into the table which do not exist, skipping existed ones.
@@ -115,7 +117,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx   The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, @Nullable Transaction tx);
+    CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously replaces an existed row associated with the same key columns values as the given one has.
@@ -124,7 +126,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Boolean> replace(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<Boolean> replace(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously replaces an expected row in the table with the given new one.
@@ -134,7 +136,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx     The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow, @Nullable Transaction tx);
+    CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously gets an existed row associated with the same key columns values as the given one has, then replaces with the given
@@ -144,7 +146,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<BinaryRow> getAndReplace(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<BinaryRow> getAndReplace(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously deletes a row with the same key columns values as the given one from the table.
@@ -153,7 +155,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx     The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Boolean> delete(BinaryRow keyRow, @Nullable Transaction tx);
+    CompletableFuture<Boolean> delete(BinaryRow keyRow, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously deletes given row from the table.
@@ -162,7 +164,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx     The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, @Nullable Transaction tx);
+    CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously gets then deletes a row with the same key columns values from the table.
@@ -171,7 +173,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx  The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<BinaryRow> getAndDelete(BinaryRow row, @Nullable Transaction tx);
+    CompletableFuture<BinaryRow> getAndDelete(BinaryRow row, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously remove rows with the same key columns values as the given one has from the table.
@@ -180,7 +182,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx   The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, @Nullable Transaction tx);
+    CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, @Nullable InternalTransaction tx);
 
     /**
      * Asynchronously remove given rows from the table.
@@ -189,8 +191,15 @@ public interface InternalTable extends AutoCloseable {
      * @param tx   The transaction.
      * @return Future representing pending completion of the operation.
      */
-    CompletableFuture<Collection<BinaryRow>> deleteAllExact(Collection<BinaryRow> rows,
-            @Nullable Transaction tx);
+    CompletableFuture<Collection<BinaryRow>> deleteAllExact(Collection<BinaryRow> rows, @Nullable InternalTransaction tx);
+
+    /**
+     * Returns a partition for a key.
+     *
+     * @param keyRow The key.
+     * @return The partition.
+     */
+    int partition(BinaryRow keyRow);
 
     /**
      * Scans given partition, providing {@link Publisher} that reactively notifies about partition rows.
@@ -199,7 +208,7 @@ public interface InternalTable extends AutoCloseable {
      * @param tx The transaction.
      * @return {@link Publisher} that reactively notifies about partition rows.
      */
-    @NotNull Publisher<BinaryRow> scan(int p, @Nullable Transaction tx);
+    @NotNull Publisher<BinaryRow> scan(int p, @Nullable InternalTransaction tx);
 
     /**
      * Gets a count of partitions of the table.
@@ -211,8 +220,8 @@ public interface InternalTable extends AutoCloseable {
     /**
      * Gets a list of current table assignments.
      *
-     * <p>Returns a list where on the i-th place resides a node id that considered as a leader for the i-th partition on the moment of
-     * invocation.
+     * <p>Returns a list where on the i-th place resides a node id that considered as a leader for
+     * the i-th partition on the moment of invocation.
      *
      * @return List of current assignments.
      */
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
index af12a94..5efdd3d 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
@@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.schema.marshaller.TupleMarshaller;
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException;
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
 import org.apache.ignite.internal.schema.row.Row;
@@ -43,277 +44,283 @@ import org.jetbrains.annotations.Nullable;
  * Key-value view implementation for binary user-object representation.
  */
 public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValueView<Tuple, Tuple> {
-    /** Marshaller. */
+    /** The marshaller. */
     private final TupleMarshallerImpl marsh;
-    
+
     /** Table manager. */
     private final TableManager tblMgr;
-    
+
     /**
-     * Constructor.
+     * The constructor.
      *
      * @param tbl       Table storage.
      * @param schemaReg Schema registry.
+     * @param tblMgr    Table manager.
+     * @param tx        The transaction.
      */
     public KeyValueBinaryViewImpl(InternalTable tbl, SchemaRegistry schemaReg, TableManager tblMgr, Transaction tx) {
         super(tbl, schemaReg, tx);
-        
+
         this.tblMgr = tblMgr;
-        
+
         marsh = new TupleMarshallerImpl(schemaReg);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple get(@NotNull Tuple key) {
         return sync(getAsync(key));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAsync(@NotNull Tuple key) {
         Objects.requireNonNull(key);
-        
+
         Row keyRow = marshal(key, null); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.get(keyRow, tx)  // Load async.
                 .thenApply(this::wrap) // Binary -> schema-aware row
                 .thenApply(TableRow::valueTuple); // Narrow to value.
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Map<Tuple, Tuple> getAll(@NotNull Collection<Tuple> keys) {
         return sync(getAllAsync(keys));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Map<Tuple, Tuple>> getAllAsync(@NotNull Collection<Tuple> keys) {
         Objects.requireNonNull(keys);
-        
+
         List<BinaryRow> keyRows = new ArrayList<>(keys.size());
-        
+
         for (Tuple keyRec : keys) {
             final Row keyRow = marshal(keyRec, null);
-            
+
             keyRows.add(keyRow);
         }
-        
+
         return tbl.getAll(keyRows, tx)
                 .thenApply(ts -> ts.stream().filter(Objects::nonNull).filter(BinaryRow::hasValue).map(this::wrap)
                         .collect(Collectors.toMap(TableRow::keyTuple, TableRow::valueTuple)));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean contains(@NotNull Tuple key) {
         return get(key) != null;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Boolean> containsAsync(@NotNull Tuple key) {
         return getAsync(key).thenApply(Objects::nonNull);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void put(@NotNull Tuple key, Tuple val) {
         sync(putAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Void> putAsync(@NotNull Tuple key, Tuple val) {
         Objects.requireNonNull(key);
-        
+
         Row row = marshal(key, val); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.upsert(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void putAll(@NotNull Map<Tuple, Tuple> pairs) {
         sync(putAllAsync(pairs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Void> putAllAsync(@NotNull Map<Tuple, Tuple> pairs) {
         Objects.requireNonNull(pairs);
-        
+
         List<BinaryRow> rows = new ArrayList<>(pairs.size());
-        
+
         for (Map.Entry<Tuple, Tuple> pair : pairs.entrySet()) {
             final Row row = marshal(pair.getKey(), pair.getValue());
-            
+
             rows.add(row);
         }
-        
+
         return tbl.upsertAll(rows, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple getAndPut(@NotNull Tuple key, Tuple val) {
         return sync(getAndPutAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAndPutAsync(@NotNull Tuple key, Tuple val) {
         Objects.requireNonNull(key);
-        
+
         Row row = marshal(key, val); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.getAndUpsert(row, tx)
                 .thenApply(this::wrap) // Binary -> schema-aware row
                 .thenApply(TableRow::valueTuple); // Narrow to value.
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean putIfAbsent(@NotNull Tuple key, @NotNull Tuple val) {
         return sync(putIfAbsentAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> putIfAbsentAsync(@NotNull Tuple key, Tuple val) {
         Objects.requireNonNull(key);
-        
+
         Row row = marshal(key, val); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.insert(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean remove(@NotNull Tuple key) {
         return sync(removeAsync(key));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean remove(@NotNull Tuple key, @NotNull Tuple val) {
         return sync(removeAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> removeAsync(@NotNull Tuple key) {
         Objects.requireNonNull(key);
-        
+
         Row row = marshal(key, null); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.delete(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> removeAsync(@NotNull Tuple key, @NotNull Tuple val) {
         Objects.requireNonNull(key);
         Objects.requireNonNull(val);
-        
+
         Row row = marshal(key, val); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.deleteExact(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Collection<Tuple> removeAll(@NotNull Collection<Tuple> keys) {
         Objects.requireNonNull(keys);
-        
+
         return sync(removeAllAsync(keys));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Collection<Tuple>> removeAllAsync(@NotNull Collection<Tuple> keys) {
         Objects.requireNonNull(keys);
-        
+
         List<BinaryRow> keyRows = new ArrayList<>(keys.size());
-        
+
         for (Tuple keyRec : keys) {
             final Row keyRow = marshal(keyRec, null);
-            
+
             keyRows.add(keyRow);
         }
-        
+
         return tbl.deleteAll(keyRows, tx)
                 .thenApply(t -> t.stream().filter(Objects::nonNull).map(this::wrap).map(TableRow::valueTuple).collect(Collectors.toList()));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple getAndRemove(@NotNull Tuple key) {
         Objects.requireNonNull(key);
-        
+
         return sync(getAndRemoveAsync(key));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAndRemoveAsync(@NotNull Tuple key) {
         Objects.requireNonNull(key);
-        
+
         return tbl.getAndDelete(marshal(key, null), tx)
                 .thenApply(this::wrap)
                 .thenApply(TableRow::valueTuple);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public boolean replace(@NotNull Tuple key, Tuple val) {
+    public boolean replace(@NotNull Tuple key, @NotNull Tuple val) {
         return sync(replaceAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public boolean replace(@NotNull Tuple key, Tuple oldVal, Tuple newVal) {
+    public boolean replace(@NotNull Tuple key, @NotNull Tuple oldVal, @NotNull Tuple newVal) {
         return sync(replaceAsync(key, oldVal, newVal));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple key, Tuple val) {
+    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple key, @NotNull Tuple val) {
         Objects.requireNonNull(key);
-        
+        Objects.requireNonNull(val);
+
         Row row = marshal(key, val); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.replace(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple key, Tuple oldVal, Tuple newVal) {
+    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple key, @NotNull Tuple oldVal, @NotNull Tuple newVal) {
         Objects.requireNonNull(key);
-        
+        Objects.requireNonNull(oldVal);
+        Objects.requireNonNull(newVal);
+
         Row oldRow = marshal(key, oldVal); // Convert to portable format to pass TX/storage layer.
         Row newRow = marshal(key, newVal); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.replace(oldRow, newRow, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public Tuple getAndReplace(@NotNull Tuple key, Tuple val) {
+    public Tuple getAndReplace(@NotNull Tuple key, @NotNull Tuple val) {
         return sync(getAndReplaceAsync(key, val));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public @NotNull CompletableFuture<Tuple> getAndReplaceAsync(@NotNull Tuple key, Tuple val) {
+    public @NotNull CompletableFuture<Tuple> getAndReplaceAsync(@NotNull Tuple key, @NotNull Tuple val) {
         Objects.requireNonNull(key);
-        
+        Objects.requireNonNull(val);
+
         return tbl.getAndReplace(marshal(key, val), tx)
                 .thenApply(this::wrap)
                 .thenApply(TableRow::valueTuple);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public <R extends Serializable> R invoke(
@@ -323,7 +330,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull <R extends Serializable> CompletableFuture<R> invokeAsync(
@@ -333,7 +340,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public <R extends Serializable> Map<Tuple, R> invokeAll(
@@ -343,7 +350,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull <R extends Serializable> CompletableFuture<Map<Tuple, R>> invokeAllAsync(
@@ -353,13 +360,22 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public KeyValueBinaryViewImpl withTransaction(Transaction tx) {
         return new KeyValueBinaryViewImpl(tbl, schemaReg, tblMgr, tx);
     }
-    
+
+    /**
+     * Returns a tuple marshaller.
+     *
+     * @return Marshaller.
+     */
+    private TupleMarshaller marshaller() {
+        return marsh;
+    }
+
     /**
      * Marshal key-value pair to a row.
      *
@@ -375,7 +391,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
             throw convertException(ex);
         }
     }
-    
+
     /**
      * Returns row.
      *
@@ -385,7 +401,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
         if (row == null) {
             return null;
         }
-        
+
         return schemaReg.resolve(row);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
index 5926d10..a905f5f 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
@@ -223,9 +223,8 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
      * {@inheritDoc}
      */
     @Override
-    public @NotNull
-    CompletableFuture<Boolean> removeAsync(@NotNull K key, @NotNull V val) {
-        BinaryRow keyRow = marshal(Objects.requireNonNull(key), val);
+    public @NotNull CompletableFuture<Boolean> removeAsync(@NotNull K key, @NotNull V val) {
+        BinaryRow keyRow = marshal(Objects.requireNonNull(key), Objects.requireNonNull(val));
         
         return tbl.deleteExact(keyRow, tx);
     }
@@ -278,7 +277,7 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
      * {@inheritDoc}
      */
     @Override
-    public boolean replace(@NotNull K key, V oldVal, V newVal) {
+    public boolean replace(@NotNull K key, @NotNull V oldVal, @NotNull V newVal) {
         return sync(replaceAsync(key, oldVal, newVal));
     }
     
@@ -286,9 +285,8 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
      * {@inheritDoc}
      */
     @Override
-    public @NotNull
-    CompletableFuture<Boolean> replaceAsync(@NotNull K key, V val) {
-        BinaryRow row = marshal(key, val);
+    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, @NotNull V val) {
+        BinaryRow row = marshal(Objects.requireNonNull(key), Objects.requireNonNull(val));
         
         return tbl.replace(row, tx);
     }
@@ -297,10 +295,11 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
      * {@inheritDoc}
      */
     @Override
-    public @NotNull
-    CompletableFuture<Boolean> replaceAsync(@NotNull K key, V oldVal, V newVal) {
-        BinaryRow oldRow = marshal(key, oldVal);
-        BinaryRow newRow = marshal(key, newVal);
+    public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, @NotNull V oldVal, @NotNull V newVal) {
+        Objects.requireNonNull(key);
+        
+        BinaryRow oldRow = marshal(key, Objects.requireNonNull(oldVal));
+        BinaryRow newRow = marshal(key, Objects.requireNonNull(newVal));
         
         return tbl.replace(oldRow, newRow, tx);
     }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
index 3ba6404..b354a0c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.internal.table;
 
+import static java.util.stream.Collectors.toList;
+
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException;
@@ -45,306 +45,264 @@ import org.jetbrains.annotations.Nullable;
 public class RecordBinaryViewImpl extends AbstractTableView implements RecordView<Tuple> {
     /** Marshaller. */
     private final TupleMarshallerImpl marsh;
-    
+
     /** Table manager. */
     private final TableManager tblMgr;
-    
+
     /**
      * Constructor.
      *
-     * @param tbl       Table.
+     * @param tbl       The table.
      * @param schemaReg Table schema registry.
      * @param tblMgr    Table manager.
      * @param tx        The transaction.
      */
     public RecordBinaryViewImpl(InternalTable tbl, SchemaRegistry schemaReg, TableManager tblMgr, @Nullable Transaction tx) {
         super(tbl, schemaReg, tx);
-        
+
         marsh = new TupleMarshallerImpl(schemaReg);
-        
+
         this.tblMgr = tblMgr;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public RecordBinaryViewImpl withTransaction(Transaction tx) {
         return new RecordBinaryViewImpl(tbl, schemaReg, tblMgr, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple get(@NotNull Tuple keyRec) {
         return sync(getAsync(keyRec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAsync(@NotNull Tuple keyRec) {
         Objects.requireNonNull(keyRec);
-        
+
         final Row keyRow = marshal(keyRec, true); // Convert to portable format to pass TX/storage layer.
-        
+
         return tbl.get(keyRow, tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Collection<Tuple> getAll(@NotNull Collection<Tuple> keyRecs) {
         return sync(getAllAsync(keyRecs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Collection<Tuple>> getAllAsync(@NotNull Collection<Tuple> keyRecs) {
         Objects.requireNonNull(keyRecs);
-        
-        HashSet<BinaryRow> keys = new HashSet<>(keyRecs.size());
-        
-        for (Tuple keyRec : keyRecs) {
-            final Row keyRow = marshal(keyRec, true);
-            
-            keys.add(keyRow);
-        }
-        
-        return tbl.getAll(keys, tx).thenApply(this::wrap);
+
+        return tbl.getAll(mapToBinary(keyRecs, true), tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void upsert(@NotNull Tuple rec) {
         sync(upsertAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Void> upsertAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.upsert(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public void upsertAll(@NotNull Collection<Tuple> recs) {
         sync(upsertAllAsync(recs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Void> upsertAllAsync(@NotNull Collection<Tuple> recs) {
         Objects.requireNonNull(recs);
-        
-        HashSet<BinaryRow> rows = new HashSet<>(recs.size());
-        
-        for (Tuple keyRec : recs) {
-            final Row row = marshal(keyRec, false);
-            
-            rows.add(row);
-        }
-        
-        return tbl.upsertAll(rows, tx);
+
+        return tbl.upsertAll(mapToBinary(recs, false), tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple getAndUpsert(@NotNull Tuple rec) {
         return sync(getAndUpsertAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAndUpsertAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.getAndUpsert(row, tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean insert(@NotNull Tuple rec) {
         return sync(insertAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> insertAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.insert(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Collection<Tuple> insertAll(@NotNull Collection<Tuple> recs) {
         return sync(insertAllAsync(recs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Collection<Tuple>> insertAllAsync(@NotNull Collection<Tuple> recs) {
         Objects.requireNonNull(recs);
-        
-        HashSet<BinaryRow> rows = new HashSet<>(recs.size());
-        
-        for (Tuple keyRec : recs) {
-            final Row row = marshal(keyRec, false);
-            
-            rows.add(row);
-        }
-        
-        return tbl.insertAll(rows, tx).thenApply(this::wrap);
+
+        return tbl.insertAll(mapToBinary(recs, false), tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean replace(@NotNull Tuple rec) {
         return sync(replaceAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean replace(@NotNull Tuple oldRec, @NotNull Tuple newRec) {
         return sync(replaceAsync(oldRec, newRec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.replace(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull Tuple oldRec, @NotNull Tuple newRec) {
         Objects.requireNonNull(oldRec);
         Objects.requireNonNull(newRec);
-        
+
         final Row oldRow = marshal(oldRec, false);
         final Row newRow = marshal(newRec, false);
-        
+
         return tbl.replace(oldRow, newRow, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple getAndReplace(@NotNull Tuple rec) {
         return sync(getAndReplaceAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAndReplaceAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.getAndReplace(row, tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean delete(@NotNull Tuple keyRec) {
         return sync(deleteAsync(keyRec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> deleteAsync(@NotNull Tuple keyRec) {
         Objects.requireNonNull(keyRec);
-        
+
         final Row keyRow = marshal(keyRec, true);
-        
+
         return tbl.delete(keyRow, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public boolean deleteExact(@NotNull Tuple rec) {
         return sync(deleteExactAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Boolean> deleteExactAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row row = marshal(rec, false);
-        
+
         return tbl.deleteExact(row, tx);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Tuple getAndDelete(@NotNull Tuple rec) {
         return sync(getAndDeleteAsync(rec));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Tuple> getAndDeleteAsync(@NotNull Tuple rec) {
         Objects.requireNonNull(rec);
-        
+
         final Row keyRow = marshal(rec, true);
-        
+
         return tbl.getAndDelete(keyRow, tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Collection<Tuple> deleteAll(@NotNull Collection<Tuple> recs) {
         return sync(deleteAllAsync(recs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull CompletableFuture<Collection<Tuple>> deleteAllAsync(@NotNull Collection<Tuple> recs) {
         Objects.requireNonNull(recs);
-        
-        HashSet<BinaryRow> keys = new HashSet<>(recs.size());
-        
-        for (Tuple keyRec : recs) {
-            final Row keyRow = marshal(keyRec, true);
-            
-            keys.add(keyRow);
-        }
-        
-        return tbl.deleteAll(keys, tx).thenApply(this::wrap);
+
+        return tbl.deleteAll(mapToBinary(recs, true), tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Collection<Tuple> deleteAllExact(@NotNull Collection<Tuple> recs) {
         return sync(deleteAllExactAsync(recs));
     }
-    
+
     /** {@inheritDoc} */
     @Override
-    public @NotNull CompletableFuture<Collection<Tuple>> deleteAllExactAsync(
-            @NotNull Collection<Tuple> recs
-    ) {
+    public @NotNull CompletableFuture<Collection<Tuple>> deleteAllExactAsync(@NotNull Collection<Tuple> recs) {
         Objects.requireNonNull(recs);
-        
-        HashSet<BinaryRow> rows = new HashSet<>(recs.size());
-        
-        for (Tuple keyRec : recs) {
-            final Row row = marshal(keyRec, false);
-            
-            rows.add(row);
-        }
-        
-        return tbl.deleteAllExact(rows, tx).thenApply(this::wrap);
+
+        return tbl.deleteAllExact(mapToBinary(recs, false), tx).thenApply(this::wrap);
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public <T extends Serializable> T invoke(
@@ -353,7 +311,7 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull <T extends Serializable> CompletableFuture<T> invokeAsync(
@@ -362,7 +320,7 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public <T extends Serializable> Map<Tuple, T> invokeAll(
@@ -371,7 +329,7 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public @NotNull <T extends Serializable> CompletableFuture<Map<Tuple, T>> invokeAllAsync(
@@ -380,11 +338,11 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
     ) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
-    
+
     /**
      * Marshal a tuple to a row.
      *
-     * @param tuple   Tuple.
+     * @param tuple   The tuple.
      * @param keyOnly Marshal key part only if {@code true}, otherwise marshal both, key and value parts.
      * @return Row.
      * @throws IgniteException If failed to marshal tuple.
@@ -400,7 +358,7 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
             throw convertException(ex);
         }
     }
-    
+
     /**
      * Returns table row tuple.
      *
@@ -410,12 +368,12 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
         if (row == null) {
             return null;
         }
-        
+
         final Row wrapped = schemaReg.resolve(row);
-        
+
         return TableRow.tuple(wrapped);
     }
-    
+
     /**
      * Returns table rows.
      *
@@ -425,15 +383,24 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
         if (rows == null) {
             return null;
         }
-    
-        Collection<BinaryRow> nonEmptyRows = rows.stream().filter(Objects::nonNull)
-                .collect(Collectors.toList());
-    
-        if (nonEmptyRows.isEmpty()) {
-            return Collections.EMPTY_LIST;
+
+        return schemaReg.resolve(rows).stream().map(TableRow::tuple).collect(toList());
+    }
+
+    /**
+     * Maps a collection of tuples to binary rows.
+     *
+     * @param rows Tuples.
+     * @param key  {@code true} to marshal only a key.
+     * @return List of binary rows.
+     */
+    private Collection<BinaryRow> mapToBinary(Collection<Tuple> rows, boolean key) {
+        Collection<BinaryRow> mapped = new ArrayList<>(rows.size());
+
+        for (Tuple row : rows) {
+            mapped.add(marshal(row, key));
         }
-    
-        return schemaReg.resolve(nonEmptyRows).stream().map(TableRow::tuple)
-                .collect(Collectors.toList());
+
+        return mapped;
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
index 03fd4b0..cd9f30b 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
@@ -17,8 +17,13 @@
 
 package org.apache.ignite.internal.table;
 
+import java.util.Objects;
 import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException;
+import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
+import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.RecordView;
@@ -26,6 +31,7 @@ import org.apache.ignite.table.Table;
 import org.apache.ignite.table.Tuple;
 import org.apache.ignite.table.mapper.Mapper;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.TestOnly;
 
 /**
  * Table view implementation for binary objects.
@@ -43,9 +49,9 @@ public class TableImpl implements Table {
     /**
      * Constructor.
      *
-     * @param tbl Table.
+     * @param tbl       The table.
      * @param schemaReg Table schema registry.
-     * @param tblMgr Table manager.
+     * @param tblMgr    Table manager.
      */
     public TableImpl(InternalTable tbl, SchemaRegistry schemaReg, TableManager tblMgr) {
         this.tbl = tbl;
@@ -82,22 +88,45 @@ public class TableImpl implements Table {
     }
 
     /** {@inheritDoc} */
-    @Override public <R> RecordView<R> recordView(Mapper<R> recMapper) {
+    @Override
+    public <R> RecordView<R> recordView(Mapper<R> recMapper) {
         return new RecordViewImpl<>(tbl, schemaReg, recMapper, null);
     }
 
     /** {@inheritDoc} */
-    @Override public RecordView<Tuple> recordView() {
+    @Override
+    public RecordView<Tuple> recordView() {
         return new RecordBinaryViewImpl(tbl, schemaReg, tblMgr, null);
     }
 
     /** {@inheritDoc} */
-    @Override public <K, V> KeyValueView<K, V> keyValueView(Mapper<K> keyMapper, Mapper<V> valMapper) {
+    @Override
+    public <K, V> KeyValueView<K, V> keyValueView(Mapper<K> keyMapper, Mapper<V> valMapper) {
         return new KeyValueViewImpl<>(tbl, schemaReg, keyMapper, valMapper, null);
     }
 
     /** {@inheritDoc} */
-    @Override public KeyValueView<Tuple, Tuple> keyValueView() {
+    @Override
+    public KeyValueView<Tuple, Tuple> keyValueView() {
         return new KeyValueBinaryViewImpl(tbl, schemaReg, tblMgr, null);
     }
+
+    /**
+     * Returns a partition for a tuple.
+     *
+     * @param t The tuple.
+     * @return The partition.
+     */
+    @TestOnly
+    public int partition(Tuple t) {
+        Objects.requireNonNull(t);
+
+        try {
+            final Row keyRow = new TupleMarshallerImpl(schemaReg).marshalKey(t);
+
+            return tbl.partition(keyRow);
+        } catch (TupleMarshallerException e) {
+            throw new IgniteInternalException(e);
+        }
+    }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
index 414a6b0..d1fa3d2 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
@@ -56,8 +56,8 @@ public class TableRow extends MutableRowTupleAdapter {
      * @param row Row.
      * @return Tuple.
      */
-    public static @NotNull Tuple tuple(@NotNull Row row) {
-        return new TableRow(row);
+    public static @Nullable Tuple tuple(@Nullable Row row) {
+        return row == null ? null : new TableRow(row);
     }
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index c7b6fc0..98e51d0 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -75,8 +75,10 @@ import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.internal.table.TableImpl;
 import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
 import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
+import org.apache.ignite.internal.table.distributed.storage.VersionedRowStore;
 import org.apache.ignite.internal.table.event.TableEvent;
 import org.apache.ignite.internal.table.event.TableEventParameters;
+import org.apache.ignite.internal.tx.TxManager;
 import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.lang.IgniteException;
@@ -124,6 +126,9 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /** Storage engine instance. Only one type is available right now, which is the {@link RocksDbStorageEngine}. */
     private final StorageEngine engine;
 
+    /** Transaction manager. */
+    private final TxManager txManager;
+
     /** Partitions store directory. */
     private final Path partitionsStoreDir;
 
@@ -153,6 +158,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param raftMgr            Raft manager.
      * @param baselineMgr        Baseline manager.
      * @param partitionsStoreDir Partitions store directory.
+     * @param txManager          TX manager.
      */
     public TableManager(
             TablesConfiguration tablesCfg,
@@ -160,13 +166,15 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             Loza raftMgr,
             BaselineManager baselineMgr,
             TopologyService topologyService,
-            Path partitionsStoreDir
+            Path partitionsStoreDir,
+            TxManager txManager
     ) {
         this.tablesCfg = tablesCfg;
         this.dataStorageCfg = dataStorageCfg;
         this.raftMgr = raftMgr;
         this.baselineMgr = baselineMgr;
         this.partitionsStoreDir = partitionsStoreDir;
+        this.txManager = txManager;
 
         netAddrResolver = addr -> {
             ClusterNode node = topologyService.getByAddress(addr);
@@ -324,7 +332,8 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                     raftGroupName(tblId, partId),
                                     newPartitionAssignment,
                                     toAdd,
-                                    () -> new PartitionListener(internalTable.storage().getOrCreatePartition(partId))
+                                    () -> new PartitionListener(tblId,
+                                            new VersionedRowStore(internalTable.storage().getOrCreatePartition(partId), txManager))
                             ).thenAccept(
                                     updatedRaftGroupService -> ((InternalTableImpl) internalTable).updateInternalTableRaftGroupService(
                                             partId, updatedRaftGroupService)
@@ -425,8 +434,8 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Creates local structures for a table.
      *
-     * @param name       Table name.
-     * @param tblId      Table id.
+     * @param name  Table name.
+     * @param tblId Table id.
      * @param assignment Affinity assignment.
      */
     private void createTableLocally(
@@ -488,7 +497,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     raftMgr.prepareRaftGroup(
                             raftGroupName(tblId, p),
                             assignment.get(p),
-                            () -> new PartitionListener(tableStorage.getOrCreatePartition(partId))
+                            () -> new PartitionListener(tblId, new VersionedRowStore(tableStorage.getOrCreatePartition(partId), txManager))
                     )
             );
         }
@@ -506,15 +515,15 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
                     partitionMap.put(p, service);
                 }
-    
-                InternalTableImpl internalTable = new InternalTableImpl(name, tblId, partitionMap,
-                        partitions, netAddrResolver, tableStorage);
-    
+
+                InternalTableImpl internalTable = new InternalTableImpl(name, tblId, partitionMap, partitions, netAddrResolver,
+                        txManager, tableStorage);
+
                 var schemaRegistry = new SchemaRegistryImpl(v -> {
                     if (!busyLock.enterBusy()) {
                         throw new IgniteException(new NodeStoppingException());
                     }
-        
+
                     try {
                         return tableSchema(tblId, v);
                     } finally {
@@ -524,7 +533,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     if (!busyLock.enterBusy()) {
                         throw new IgniteException(new NodeStoppingException());
                     }
-        
+
                     try {
                         return latestSchemaVersion(tblId);
                     } finally {
@@ -553,7 +562,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Return table schema of certain version from history.
      *
-     * @param tblId Table id.
+     * @param tblId     Table id.
      * @param schemaVer Schema version.
      * @return Schema descriptor.
      */
@@ -583,7 +592,8 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     return false;
                 }
 
-                @Override public void remove(@NotNull Throwable exception) {
+                @Override
+                public void remove(@NotNull Throwable exception) {
                     fur.completeExceptionally(exception);
                 }
             };
@@ -608,10 +618,11 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * Gets a schema descriptor from the local node configuration storage.
      *
      * @param schemaVer Schema version.
-     * @param tblCfg Table configuration.
+     * @param tblCfg    Table configuration.
      * @return Schema descriptor.
      */
-    @NotNull private SchemaDescriptor getSchemaDescriptorLocally(int schemaVer, ExtendedTableConfiguration tblCfg) {
+    @NotNull
+    private SchemaDescriptor getSchemaDescriptorLocally(int schemaVer, ExtendedTableConfiguration tblCfg) {
         SchemaConfiguration schemaCfg = tblCfg.schemas().get(String.valueOf(schemaVer));
 
         assert schemaCfg != null;
@@ -622,8 +633,8 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Drops local structures for a table.
      *
-     * @param name       Table name.
-     * @param tblId      Table id.
+     * @param name  Table name.
+     * @param tblId Table id.
      * @param assignment Affinity assignment.
      */
     private void dropTableLocally(String name, IgniteUuid tblId, List<List<ClusterNode>> assignment) {
@@ -652,7 +663,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Compounds a RAFT group unique name.
      *
-     * @param tblId     Table identifier.
+     * @param tblId Table identifier.
      * @param partition Number of table partitions.
      * @return A RAFT group name.
      */
@@ -683,10 +694,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Creates a new table with the specified name or returns an existing table with the same name.
      *
-     * @param name               Table name.
-     * @param tableInitChange    Table configuration.
-     * @param exceptionWhenExist If the value is {@code true}, an exception will be thrown when the table already exists, {@code false}
-     *                           means the existing table will be returned.
+     * @param name Table name.
+     * @param tableInitChange Table configuration.
+     * @param exceptionWhenExist If the value is {@code true}, an exception will be thrown when the table already exists, {@code
+     *         false} means the existing table will be returned.
      * @return A table instance.
      */
     private CompletableFuture<Table> createTableAsync(
@@ -826,7 +837,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     /**
      * Internal method for creating table asynchronously.
      *
-     * @param name        Table name.
+     * @param name Table name.
      * @param tableChange Table cahnger.
      * @return Future representing pending completion of the operation.
      */
@@ -1053,11 +1064,11 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private List<String> tableNamesConfigured() {
         return ConfigurationUtil.directValue(tablesCfg.tables()).namedListKeys();
     }
-    
+
     /**
      * Checks that the schema is configured in the Metasorage consensus.
      *
-     * @param tblId     Table id.
+     * @param tblId Table id.
      * @param schemaVer Schema version.
      * @return True when the schema configured, false otherwise.
      */
@@ -1065,7 +1076,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private boolean isSchemaExists(IgniteUuid tblId, int schemaVer) {
         return latestSchemaVersion(tblId) >= schemaVer;
     }
-    
+
     /**
      * Gets the latest version of the table schema which available in Metastore.
      *
@@ -1075,34 +1086,34 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private int latestSchemaVersion(IgniteUuid tblId) {
         NamedListView<TableView> directTablesCfg = ((DirectConfigurationProperty<NamedListView<TableView>>) tablesCfg
                 .tables()).directValue();
-        
+
         ExtendedTableView viewForId = null;
-        
+
         // TODO: IGNITE-15721 Need to review this approach after the ticket would be fixed.
         // Probably, it won't be required getting configuration of all tables from Metastor.
         for (String name : directTablesCfg.namedListKeys()) {
             ExtendedTableView tblView = (ExtendedTableView) directTablesCfg.get(name);
-            
+
             if (tblView != null && tblId.equals(IgniteUuid.fromString(tblView.id()))) {
                 viewForId = tblView;
-                
+
                 break;
             }
         }
-        
+
         int lastVer = INITIAL_SCHEMA_VERSION;
-        
+
         for (String schemaVerAsStr : viewForId.schemas().namedListKeys()) {
             int ver = Integer.parseInt(schemaVerAsStr);
-            
+
             if (ver > lastVer) {
                 lastVer = ver;
             }
         }
-        
+
         return lastVer;
     }
-    
+
     /** {@inheritDoc} */
     @Override
     public Table table(String name) {
@@ -1428,7 +1439,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      *
      * @param oldAssignments Old assignment.
      * @param newAssignments New assignment.
-     * @param tblId          Table ID.
+     * @param tblId Table ID.
      * @return Future, which completes, when update finished.
      */
     private CompletableFuture<Void> updateRaftTopology(
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableTxManagerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableTxManagerImpl.java
new file mode 100644
index 0000000..f1f34e3
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableTxManagerImpl.java
@@ -0,0 +1,56 @@
+/*
+ * 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.table.distributed;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.table.distributed.command.FinishTxCommand;
+import org.apache.ignite.internal.tx.LockManager;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.tx.impl.TxManagerImpl;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.raft.jraft.RaftMessagesFactory;
+import org.apache.ignite.raft.jraft.rpc.ActionRequest;
+
+/**
+ * Transaction manager extension for Ignite tables.
+ *
+ * <p>Uses raft protocol to replicate tx finish state for a partition group.
+ */
+public class TableTxManagerImpl extends TxManagerImpl {
+    private static final RaftMessagesFactory FACTORY = new RaftMessagesFactory();
+
+    private static final int FINISH_TIMEOUT = 1000;
+
+    /**
+     * The constructor.
+     *
+     * @param clusterService Cluster service.
+     * @param lockManager    Lock manager.
+     */
+    public TableTxManagerImpl(ClusterService clusterService, LockManager lockManager) {
+        super(clusterService, lockManager);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected CompletableFuture<?> finish(String groupId, Timestamp ts, boolean commit) {
+        ActionRequest req = FACTORY.actionRequest().command(new FinishTxCommand(ts, commit)).groupId(groupId).readOnlySafe(true).build();
+
+        return clusterService.messagingService().invoke(clusterService.topologyService().localMember(), req, FINISH_TIMEOUT);
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/CommandUtils.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/CommandUtils.java
index 12cba46..b1adbf8 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/CommandUtils.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/CommandUtils.java
@@ -25,6 +25,7 @@ import java.util.Collection;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.jetbrains.annotations.Nullable;
 
@@ -41,34 +42,32 @@ public class CommandUtils {
      * Writes a list of rows to byte array.
      *
      * @param rows     Collection of rows.
-     * @param consumer Byte array consumer.
+     * @return         Rows data.
      */
-    public static void rowsToBytes(Collection<BinaryRow> rows, Consumer<byte[]> consumer) {
+    public static byte[] rowsToBytes(Collection<BinaryRow> rows) {
         if (rows == null || rows.isEmpty()) {
-            return;
+            return null;
         }
 
         try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
             for (BinaryRow row : rows) {
-                rowToBytes(row, bytes -> {
-                    try {
-                        baos.write(intToBytes(bytes.length));
-
-                        baos.write(bytes);
-                    } catch (IOException e) {
-                        LOG.error("Could not write row to stream [row=" + row + ']', e);
-                    }
-
-                });
+                if (row == null) {
+                    baos.write(intToBytes(0));
+                } else {
+                    byte[] bytes = rowToBytes(row);
+
+                    baos.write(intToBytes(bytes.length));
+                    baos.write(bytes);
+                }
             }
 
             baos.flush();
 
-            consumer.accept(baos.toByteArray());
+            return baos.toByteArray();
         } catch (IOException e) {
             LOG.error("Could not write rows to stream [rows=" + rows.size() + ']', e);
 
-            consumer.accept(null);
+            throw new IgniteInternalException(e);
         }
     }
 
@@ -76,11 +75,11 @@ public class CommandUtils {
      * Writes a row to byte array.
      *
      * @param row      Row.
-     * @param consumer Byte array consumer.
+     * @return         Row bytes.
      */
-    public static void rowToBytes(@Nullable BinaryRow row, Consumer<byte[]> consumer) {
+    public static byte[] rowToBytes(@Nullable BinaryRow row) {
         if (row == null) {
-            return;
+            return null;
         }
 
         try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
@@ -88,11 +87,11 @@ public class CommandUtils {
 
             baos.flush();
 
-            consumer.accept(baos.toByteArray());
+            return baos.toByteArray();
         } catch (IOException e) {
             LOG.error("Could not write row to stream [row=" + row + ']', e);
 
-            consumer.accept(null);
+            throw new IgniteInternalException(e);
         }
     }
 
@@ -119,7 +118,11 @@ public class CommandUtils {
 
                 int len = bytesToInt(lenBytes);
 
-                assert len > 0;
+                if (len == 0) {
+                    consumer.accept(null);
+
+                    continue;
+                }
 
                 rowBytes = new byte[len];
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteAllCommand.java
index d177715..65e6f5c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteAllCommand.java
@@ -17,52 +17,26 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command deletes entries by the passed keys.
  */
-public class DeleteAllCommand implements WriteCommand {
-    /** Binary rows. */
-    private transient Set<BinaryRow> rows;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowsBytes;
-
+public class DeleteAllCommand extends MultiKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of DeleteAllCommand with the given set of keys to be deleted. The {@code keyRows} should not be {@code null}
      * or empty.
      *
-     * @param keyRows Collection of binary row keys to be deleted.
-     */
-    public DeleteAllCommand(@NotNull Set<BinaryRow> keyRows) {
-        assert keyRows != null && !keyRows.isEmpty();
-
-        this.rows = keyRows;
-
-        CommandUtils.rowsToBytes(keyRows, bytes -> rowsBytes = bytes);
-    }
-
-    /**
-     * Returns a set of binary key rows to be deleted.
+     * @param keyRows   Collection of binary row keys to be deleted.
+     * @param timestamp The timestamp.
      *
-     * @return Binary keys.
+     * @see TransactionalCommand
      */
-    public Set<BinaryRow> getRows() {
-        if (rows == null && rowsBytes != null) {
-            rows = new HashSet<>();
-
-            CommandUtils.readRows(rowsBytes, rows::add);
-        }
-
-        return rows;
+    public DeleteAllCommand(@NotNull Collection<BinaryRow> keyRows, @NotNull Timestamp timestamp) {
+        super(keyRows, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java
index cadcd03..cf56df7 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java
@@ -18,48 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command deletes a entry by passed key.
  */
-public class DeleteCommand implements WriteCommand {
-    /** Binary key row. */
-    private transient BinaryRow keyRow;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] keyRowBytes;
-
+public class DeleteCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of DeleteCommand with the given key to be deleted. The {@code keyRow} should not be {@code null}.
      *
-     * @param keyRow Binary key row.
-     */
-    public DeleteCommand(@NotNull BinaryRow keyRow) {
-        assert keyRow != null;
-
-        this.keyRow = keyRow;
-
-        CommandUtils.rowToBytes(keyRow, bytes -> keyRowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary key row to be deleted.
+     * @param keyRow    Binary key row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary key.
+     * @see TransactionalCommand
      */
-    public BinaryRow getKeyRow() {
-        if (keyRow == null) {
-            keyRow = new ByteBufferRow(keyRowBytes);
-        }
-
-        return keyRow;
+    public DeleteCommand(@NotNull BinaryRow keyRow, @NotNull Timestamp timestamp) {
+        super(keyRow, timestamp);
     }
-
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java
index eb45e9c..b292040 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java
@@ -17,52 +17,26 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command deletes entries that exact the same as the rows passed.
  */
-public class DeleteExactAllCommand implements WriteCommand {
-    /** Binary rows. */
-    private transient Set<BinaryRow> rows;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowsBytes;
-
+public class DeleteExactAllCommand extends MultiKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of DeleteExactAllCommand with the given set of rows to be deleted. The {@code rows} should not be {@code null}
      * or empty.
      *
-     * @param rows Binary rows.
-     */
-    public DeleteExactAllCommand(@NotNull Set<BinaryRow> rows) {
-        assert rows != null && !rows.isEmpty();
-
-        this.rows = rows;
-
-        CommandUtils.rowsToBytes(rows, bytes -> rowsBytes = bytes);
-    }
-
-    /**
-     * Gets a set of binary rows to be deleted.
+     * @param rows      Binary rows.
+     * @param timestamp The timestamp.
      *
-     * @return Binary rows.
+     * @see TransactionalCommand
      */
-    public Set<BinaryRow> getRows() {
-        if (rows == null && rowsBytes != null) {
-            rows = new HashSet<>();
-
-            CommandUtils.readRows(rowsBytes, rows::add);
-        }
-
-        return rows;
+    public DeleteExactAllCommand(@NotNull Collection<BinaryRow> rows, @NotNull Timestamp timestamp) {
+        super(rows, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactCommand.java
index 232a846..b6cf02c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactCommand.java
@@ -18,48 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command deletes an entry that is exact the same as the row passed.
  */
-public class DeleteExactCommand implements WriteCommand {
-    /** Binary row. */
-    private transient BinaryRow row;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
+public class DeleteExactCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of DeleteExactCommand with the given row to be deleted. The {@code row} should not be {@code null}.
      *
-     * @param row Binary row.
-     */
-    public DeleteExactCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.row = row;
-
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary key row to be got.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
-
-        return row;
+    public DeleteExactCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
-
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/FinishTxCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/FinishTxCommand.java
new file mode 100644
index 0000000..fa14227
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/FinishTxCommand.java
@@ -0,0 +1,59 @@
+/*
+ * 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.table.distributed.command;
+
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.raft.client.WriteCommand;
+
+/** State machine command to finish a transaction. */
+public class FinishTxCommand implements WriteCommand {
+    /** The timestamp. */
+    private final Timestamp timestamp;
+
+    /** Commit or rollback state. */
+    private final boolean finish;
+
+    /**
+     * The constructor.
+     *
+     * @param timestamp The timestamp.
+     * @param finish    Commit or rollback state {@code True} to commit.
+     */
+    public FinishTxCommand(Timestamp timestamp, boolean finish) {
+        this.timestamp = timestamp;
+        this.finish = finish;
+    }
+
+    /**
+     * Returns a timestamp.
+     *
+     * @return The timestamp.
+     */
+    public Timestamp timestamp() {
+        return timestamp;
+    }
+
+    /**
+     * Returns commit or rollback state.
+     *
+     * @return Commit or rollback state.
+     */
+    public boolean finish() {
+        return finish;
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAllCommand.java
index 19ebed3..70b09f2 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAllCommand.java
@@ -17,51 +17,25 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * This is a command for the batch get operation.
  */
-public class GetAllCommand implements ReadCommand {
-    /** Binary key rows. */
-    private transient Set<BinaryRow> keyRows;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] keyRowsBytes;
-
+public class GetAllCommand extends MultiKeyCommand implements ReadCommand {
     /**
      * Creates a new instance of GetAllCommand with the given keys to be got. The {@code keyRows} should not be {@code null} or empty.
      *
-     * @param keyRows Binary key rows.
-     */
-    public GetAllCommand(@NotNull Set<BinaryRow> keyRows) {
-        assert keyRows != null && !keyRows.isEmpty();
-
-        this.keyRows = keyRows;
-
-        CommandUtils.rowsToBytes(keyRows, bytes -> keyRowsBytes = bytes);
-    }
-
-    /**
-     * Gets a set of binary key rows to be got.
+     * @param keyRows   Binary key rows.
+     * @param timestamp The timestamp.
      *
-     * @return Binary keys.
+     * @see TransactionalCommand
      */
-    public Set<BinaryRow> getKeyRows() {
-        if (keyRows == null && keyRowsBytes != null) {
-            keyRows = new HashSet<>();
-
-            CommandUtils.readRows(keyRowsBytes, keyRows::add);
-        }
-
-        return keyRows;
+    public GetAllCommand(@NotNull Collection<BinaryRow> keyRows, @NotNull Timestamp timestamp) {
+        super(keyRows, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndDeleteCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndDeleteCommand.java
index 189c90a..e79e9a7 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndDeleteCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndDeleteCommand.java
@@ -18,48 +18,24 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * This is a command to get a value before delete it.
  */
-public class GetAndDeleteCommand implements WriteCommand {
-    /** Binary key row. */
-    private transient BinaryRow keyRow;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] keyRowBytes;
-
+public class GetAndDeleteCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of GetAndDeleteCommand with the given key to be got and deleted. The {@code keyRow} should not be {@code
      * null}.
      *
-     * @param keyRow Binary key row.
-     */
-    public GetAndDeleteCommand(@NotNull BinaryRow keyRow) {
-        assert keyRow != null;
-
-        this.keyRow = keyRow;
-
-        CommandUtils.rowToBytes(keyRow, bytes -> keyRowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary key row to be got and deleted.
+     * @param keyRow    Binary key row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary key.
+     * @see TransactionalCommand
      */
-    public BinaryRow getKeyRow() {
-        if (keyRow == null) {
-            keyRow = new ByteBufferRow(keyRowBytes);
-        }
-
-        return keyRow;
+    public GetAndDeleteCommand(@NotNull BinaryRow keyRow, @NotNull Timestamp timestamp) {
+        super(keyRow, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndReplaceCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndReplaceCommand.java
index db5e82f..2c80a97 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndReplaceCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndReplaceCommand.java
@@ -18,48 +18,24 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * This is a command to get a value before replace it.
  */
-public class GetAndReplaceCommand implements WriteCommand {
-    /** Binary row. */
-    private transient BinaryRow row;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
+public class GetAndReplaceCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of GetAndReplaceCommand with the given row to be got and replaced. The {@code row} should not be {@code
      * null}.
      *
-     * @param row Binary row.
-     */
-    public GetAndReplaceCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.row = row;
-
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary row to be got and replaced.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
-
-        return row;
+    public GetAndReplaceCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndUpsertCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndUpsertCommand.java
index 8875e67..35e4ad9 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndUpsertCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetAndUpsertCommand.java
@@ -18,47 +18,24 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * This is a command to get a value before upsert it.
  */
-public class GetAndUpsertCommand implements WriteCommand {
-    /** Binary key row. */
-    private transient BinaryRow keyRow;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] keyRowBytes;
-
+public class GetAndUpsertCommand extends SingleKeyCommand implements WriteCommand {
     /**
-     * Creates a new instance of GetAndUpsertCommand with the given row to be got and upserted. The {@code row} should not be {@code null}.
+     * Creates a new instance of GetAndUpsertCommand with the given row to be got and upserted. The
+     * {@code row} should not be {@code null}.
      *
-     * @param row Binary row.
-     */
-    public GetAndUpsertCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.keyRow = row;
-
-        CommandUtils.rowToBytes(row, bytes -> keyRowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary key row to be got and upserted.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary key.
+     * @see TransactionalCommand
      */
-    public BinaryRow getKeyRow() {
-        if (keyRow == null) {
-            keyRow = new ByteBufferRow(keyRowBytes);
-        }
-
-        return keyRow;
+    public GetAndUpsertCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetCommand.java
index 4d84d40..a1dc5b6 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/GetCommand.java
@@ -18,47 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command gets a value by key specified.
  */
-public class GetCommand implements ReadCommand {
-    /** Binary key row. */
-    private transient BinaryRow keyRow;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] keyRowBytes;
-
+public class GetCommand extends SingleKeyCommand implements ReadCommand {
     /**
      * Creates a new instance of GetCommand with the given key to be got. The {@code keyRow} should not be {@code null}.
      *
-     * @param keyRow Binary key row.
-     */
-    public GetCommand(@NotNull BinaryRow keyRow) {
-        assert keyRow != null;
-
-        this.keyRow = keyRow;
-
-        CommandUtils.rowToBytes(keyRow, bytes -> keyRowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary key row to be got.
+     * @param keyRow    Binary key row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary key.
+     * @see TransactionalCommand
      */
-    public BinaryRow getKeyRow() {
-        if (keyRow == null) {
-            keyRow = new ByteBufferRow(keyRowBytes);
-        }
-
-        return keyRow;
+    public GetCommand(@NotNull BinaryRow keyRow, @NotNull Timestamp timestamp) {
+        super(keyRow, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertAllCommand.java
index a58b8ae..2ef0317 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertAllCommand.java
@@ -17,51 +17,25 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command inserts a batch rows.
  */
-public class InsertAllCommand implements WriteCommand {
-    /** Binary rows. */
-    private transient Set<BinaryRow> rows;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowsBytes;
-
+public class InsertAllCommand extends MultiKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of InsertAllCommand with the given rows to be inserted. The {@code rows} should not be {@code null} or empty.
      *
-     * @param rows Binary rows.
-     */
-    public InsertAllCommand(@NotNull Set<BinaryRow> rows) {
-        assert rows != null && !rows.isEmpty();
-
-        this.rows = rows;
-
-        CommandUtils.rowsToBytes(rows, bytes -> rowsBytes = bytes);
-    }
-
-    /**
-     * Gets a set of binary rows to be inserted.
+     * @param rows        Binary rows.
+     * @param timestamp   The timestamp.
      *
-     * @return Binary rows.
+     * @see TransactionalCommand
      */
-    public Set<BinaryRow> getRows() {
-        if (rows == null && rowsBytes != null) {
-            rows = new HashSet<>();
-
-            CommandUtils.readRows(rowsBytes, rows::add);
-        }
-
-        return rows;
+    public InsertAllCommand(@NotNull Collection<BinaryRow> rows, @NotNull Timestamp timestamp) {
+        super(rows, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertCommand.java
index b4290a5..8e32117 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/InsertCommand.java
@@ -18,47 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command inserts a row.
  */
-public class InsertCommand implements WriteCommand {
-    /** Binary row. */
-    private transient BinaryRow row;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
+public class InsertCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of InsertCommand with the given row to be inserted. The {@code row} should not be {@code null}.
      *
-     * @param row Binary row.
-     */
-    public InsertCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.row = row;
-
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary row to be inserted.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
-
-        return row;
+    public InsertCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/MultiKeyCommand.java
similarity index 61%
copy from modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/MultiKeyCommand.java
index eb45e9c..9911b93 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteExactAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/MultiKeyCommand.java
@@ -17,18 +17,22 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * The command deletes entries that exact the same as the rows passed.
+ * A multi key transactional command.
  */
-public class DeleteExactAllCommand implements WriteCommand {
+public abstract class MultiKeyCommand implements TransactionalCommand, Serializable {
     /** Binary rows. */
-    private transient Set<BinaryRow> rows;
+    private transient Collection<BinaryRow> rows;
+
+    /** The timestamp. */
+    private @NotNull Timestamp timestamp;
 
     /*
      * Row bytes.
@@ -38,31 +42,42 @@ public class DeleteExactAllCommand implements WriteCommand {
     private byte[] rowsBytes;
 
     /**
-     * Creates a new instance of DeleteExactAllCommand with the given set of rows to be deleted. The {@code rows} should not be {@code null}
-     * or empty.
+     * The constructor.
      *
-     * @param rows Binary rows.
+     * @param rows Rows.
+     * @param ts   The timestamp.
      */
-    public DeleteExactAllCommand(@NotNull Set<BinaryRow> rows) {
+    public MultiKeyCommand(@NotNull Collection<BinaryRow> rows, @NotNull Timestamp ts) {
         assert rows != null && !rows.isEmpty();
-
         this.rows = rows;
+        this.timestamp = ts;
 
-        CommandUtils.rowsToBytes(rows, bytes -> rowsBytes = bytes);
+        rowsBytes = CommandUtils.rowsToBytes(rows);
     }
 
     /**
-     * Gets a set of binary rows to be deleted.
+     * Gets a collection of binary rows.
      *
      * @return Binary rows.
      */
-    public Set<BinaryRow> getRows() {
+    public Collection<BinaryRow> getRows() {
         if (rows == null && rowsBytes != null) {
-            rows = new HashSet<>();
+            rows = new ArrayList<>();
 
             CommandUtils.readRows(rowsBytes, rows::add);
         }
 
         return rows;
     }
+
+    /**
+     * Returns a timestamp.
+     *
+     * @return The timestamp.
+     */
+    @NotNull
+    @Override
+    public Timestamp getTimestamp() {
+        return timestamp;
+    }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceCommand.java
index 57c8e87..760c56f 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceCommand.java
@@ -19,26 +19,17 @@ package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command replaces an old entry to a new one.
  */
-public class ReplaceCommand implements WriteCommand {
-    /** Replacing binary row. */
-    private transient BinaryRow row;
-
+public class ReplaceCommand extends SingleKeyCommand implements WriteCommand {
     /** Replaced binary row. */
     private transient BinaryRow oldRow;
 
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
     /**
      * Old row bytes.
      * TODO: Remove the field after (IGNITE-14793).
@@ -46,34 +37,20 @@ public class ReplaceCommand implements WriteCommand {
     private byte[] oldRowBytes;
 
     /**
-     * Creates a new instance of ReplaceCommand with the given two rows to be replaced each other. Both rows should not be {@code null}.
+     * Creates a new instance of ReplaceCommand with the given two rows to be replaced each other.
+     * Both rows should not be {@code null}.
      *
-     * @param oldRow Old Binary row.
-     * @param row    Binary row.
-     */
-    public ReplaceCommand(@NotNull BinaryRow oldRow, @NotNull BinaryRow row) {
-        assert oldRow != null;
-        assert row != null;
-
-        this.oldRow = oldRow;
-        this.row = row;
-
-        CommandUtils.rowToBytes(oldRow, bytes -> oldRowBytes = bytes);
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary row which will be after replace.
+     * @param oldRow        Old Binary row.
+     * @param row           Binary row.
+     * @param timestamp     The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    @NotNull
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
+    public ReplaceCommand(@NotNull BinaryRow oldRow, @NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
+        assert oldRow != null;
 
-        return row;
+        oldRowBytes = CommandUtils.rowToBytes(oldRow);
     }
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceIfExistCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceIfExistCommand.java
index abb47ae..a2e0cc6 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceIfExistCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/ReplaceIfExistCommand.java
@@ -18,46 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command replaces an old entry to a new one.
  */
-public class ReplaceIfExistCommand implements WriteCommand {
-    /** Binary row. */
-    private transient BinaryRow row;
-
-    /**
-     * Row bytes. It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
+public class ReplaceIfExistCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of ReplaceIfExistCommand with the given row to be replaced. The {@code row} should not be {@code null}.
      *
-     * @param row Binary row.
-     */
-    public ReplaceIfExistCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.row = row;
-
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary row to be replaced.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
-
-        return row;
+    public ReplaceIfExistCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/SingleKeyCommand.java
similarity index 68%
copy from modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/SingleKeyCommand.java
index cadcd03..5953915 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/DeleteCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/SingleKeyCommand.java
@@ -17,18 +17,22 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
+import java.io.Serializable;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
-import org.apache.ignite.raft.client.WriteCommand;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * The command deletes a entry by passed key.
+ * A single key transactional command.
  */
-public class DeleteCommand implements WriteCommand {
+public abstract class SingleKeyCommand implements TransactionalCommand, Serializable {
     /** Binary key row. */
     private transient BinaryRow keyRow;
 
+    /** The timestamp. */
+    private @NotNull final Timestamp timestamp;
+
     /*
      * Row bytes.
      * It is a temporary solution, before network have not implement correct serialization BinaryRow.
@@ -37,16 +41,18 @@ public class DeleteCommand implements WriteCommand {
     private byte[] keyRowBytes;
 
     /**
-     * Creates a new instance of DeleteCommand with the given key to be deleted. The {@code keyRow} should not be {@code null}.
+     * The constructor.
      *
-     * @param keyRow Binary key row.
+     * @param keyRow    The row.
+     * @param timestamp The timestamp.
      */
-    public DeleteCommand(@NotNull BinaryRow keyRow) {
+    public SingleKeyCommand(@NotNull BinaryRow keyRow, @NotNull Timestamp timestamp) {
         assert keyRow != null;
 
         this.keyRow = keyRow;
+        this.timestamp = timestamp;
 
-        CommandUtils.rowToBytes(keyRow, bytes -> keyRowBytes = bytes);
+        keyRowBytes = CommandUtils.rowToBytes(keyRow);
     }
 
     /**
@@ -54,7 +60,7 @@ public class DeleteCommand implements WriteCommand {
      *
      * @return Binary key.
      */
-    public BinaryRow getKeyRow() {
+    public BinaryRow getRow() {
         if (keyRow == null) {
             keyRow = new ByteBufferRow(keyRowBytes);
         }
@@ -62,4 +68,14 @@ public class DeleteCommand implements WriteCommand {
         return keyRow;
     }
 
+    /**
+     * Returns a timestamp.
+     *
+     * @return The timestamp.
+     */
+    @NotNull
+    @Override
+    public Timestamp getTimestamp() {
+        return timestamp;
+    }
 }
diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/Waiter.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TransactionalCommand.java
similarity index 63%
copy from modules/transactions/src/main/java/org/apache/ignite/internal/tx/Waiter.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TransactionalCommand.java
index 3146f95..8a45425 100644
--- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/Waiter.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TransactionalCommand.java
@@ -15,19 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.tx;
+package org.apache.ignite.internal.table.distributed.command;
+
+import org.apache.ignite.internal.tx.Timestamp;
+import org.jetbrains.annotations.NotNull;
 
 /**
- * The waiter.
+ * A marker interface for a transactional command.
+ *
+ * <p>Each transactional command contains a timestamp, bounding it to a specific transaction.
+ *
+ * @see Timestamp
  */
-public interface Waiter {
-    /**
-     * Returns associated timestamp.
-     */
-    Timestamp timestamp();
-
+public interface TransactionalCommand {
     /**
-     * Returns {@code true} if the waiter holds the lock.
+     * Returns a timestamp.
+     *
+     * @return The timestamp.
      */
-    boolean locked();
+    @NotNull
+    public Timestamp getTimestamp();
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertAllCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertAllCommand.java
index 47cef67..9c4f95c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertAllCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertAllCommand.java
@@ -17,51 +17,25 @@
 
 package org.apache.ignite.internal.table.distributed.command;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.Collection;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command puts a batch rows.
  */
-public class UpsertAllCommand implements WriteCommand {
-    /** Binary rows. */
-    private transient Set<BinaryRow> rows;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowsBytes;
-
+public class UpsertAllCommand extends MultiKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of UpsertAllCommand with the given rows to be upserted. The {@code rows} should not be {@code null} or empty.
      *
-     * @param rows Binary rows.
-     */
-    public UpsertAllCommand(@NotNull Set<BinaryRow> rows) {
-        assert rows != null && !rows.isEmpty();
-
-        this.rows = rows;
-
-        CommandUtils.rowsToBytes(rows, bytes -> rowsBytes = bytes);
-    }
-
-    /**
-     * Gets a set of binary rows to be upserted.
+     * @param rows      Binary rows.
+     * @param timestamp The timestamp.
      *
-     * @return Binary rows.
+     * @see TransactionalCommand
      */
-    public Set<BinaryRow> getRows() {
-        if (rows == null && rowsBytes != null) {
-            rows = new HashSet<>();
-
-            CommandUtils.readRows(rowsBytes, rows::add);
-        }
-
-        return rows;
+    public UpsertAllCommand(@NotNull Collection<BinaryRow> rows, @NotNull Timestamp timestamp) {
+        super(rows, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertCommand.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertCommand.java
index 1c2dba7..093cc80 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertCommand.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/UpsertCommand.java
@@ -18,47 +18,23 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * The command inserts or updates a value for the key specified.
  */
-public class UpsertCommand implements WriteCommand {
-    /** Binary row. */
-    private transient BinaryRow row;
-
-    /*
-     * Row bytes.
-     * It is a temporary solution, before network have not implement correct serialization BinaryRow.
-     * TODO: Remove the field after (IGNITE-14793).
-     */
-    private byte[] rowBytes;
-
+public class UpsertCommand extends SingleKeyCommand implements WriteCommand {
     /**
      * Creates a new instance of UpsertCommand with the given row to be upserted. The {@code row} should not be {@code null}.
      *
-     * @param row Binary row.
-     */
-    public UpsertCommand(@NotNull BinaryRow row) {
-        assert row != null;
-
-        this.row = row;
-
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
-    }
-
-    /**
-     * Gets a binary row to be upserted.
+     * @param row       Binary row.
+     * @param timestamp The timestamp.
      *
-     * @return Binary row.
+     * @see TransactionalCommand
      */
-    public BinaryRow getRow() {
-        if (row == null) {
-            row = new ByteBufferRow(rowBytes);
-        }
-
-        return row;
+    public UpsertCommand(@NotNull BinaryRow row, @NotNull Timestamp timestamp) {
+        super(row, timestamp);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/MultiRowsResponse.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/MultiRowsResponse.java
index 4fcc806..55c52fa 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/MultiRowsResponse.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/MultiRowsResponse.java
@@ -54,7 +54,7 @@ public class MultiRowsResponse implements Serializable {
     public MultiRowsResponse(List<BinaryRow> rows) {
         this.rows = rows;
 
-        CommandUtils.rowsToBytes(rows, bytes -> rowsBytes = bytes);
+        rowsBytes = CommandUtils.rowsToBytes(rows);
     }
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/SingleRowResponse.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/SingleRowResponse.java
index 4ed7fe6..2ced864 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/SingleRowResponse.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/response/SingleRowResponse.java
@@ -54,7 +54,7 @@ public class SingleRowResponse implements Serializable {
     public SingleRowResponse(@Nullable BinaryRow row) {
         this.row = row;
 
-        CommandUtils.rowToBytes(row, bytes -> rowBytes = bytes);
+        rowBytes = CommandUtils.rowToBytes(row);
     }
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java
index 70685b9..fbcf3b3 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java
@@ -17,33 +17,27 @@
 
 package org.apache.ignite.internal.table.distributed.raft;
 
-import java.nio.ByteBuffer;
+import static org.apache.ignite.lang.LoggerMessageHelper.format;
+
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Consumer;
-import java.util.stream.Collectors;
 import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.ByteBufferRow;
 import org.apache.ignite.internal.storage.DataRow;
-import org.apache.ignite.internal.storage.PartitionStorage;
-import org.apache.ignite.internal.storage.SearchRow;
 import org.apache.ignite.internal.storage.StorageException;
-import org.apache.ignite.internal.storage.basic.DeleteExactInvokeClosure;
-import org.apache.ignite.internal.storage.basic.GetAndRemoveInvokeClosure;
-import org.apache.ignite.internal.storage.basic.GetAndReplaceInvokeClosure;
-import org.apache.ignite.internal.storage.basic.InsertInvokeClosure;
-import org.apache.ignite.internal.storage.basic.ReplaceExactInvokeClosure;
 import org.apache.ignite.internal.storage.basic.SimpleDataRow;
 import org.apache.ignite.internal.table.distributed.command.DeleteAllCommand;
 import org.apache.ignite.internal.table.distributed.command.DeleteCommand;
 import org.apache.ignite.internal.table.distributed.command.DeleteExactAllCommand;
 import org.apache.ignite.internal.table.distributed.command.DeleteExactCommand;
+import org.apache.ignite.internal.table.distributed.command.FinishTxCommand;
 import org.apache.ignite.internal.table.distributed.command.GetAllCommand;
 import org.apache.ignite.internal.table.distributed.command.GetAndDeleteCommand;
 import org.apache.ignite.internal.table.distributed.command.GetAndReplaceCommand;
@@ -51,8 +45,11 @@ import org.apache.ignite.internal.table.distributed.command.GetAndUpsertCommand;
 import org.apache.ignite.internal.table.distributed.command.GetCommand;
 import org.apache.ignite.internal.table.distributed.command.InsertAllCommand;
 import org.apache.ignite.internal.table.distributed.command.InsertCommand;
+import org.apache.ignite.internal.table.distributed.command.MultiKeyCommand;
 import org.apache.ignite.internal.table.distributed.command.ReplaceCommand;
 import org.apache.ignite.internal.table.distributed.command.ReplaceIfExistCommand;
+import org.apache.ignite.internal.table.distributed.command.SingleKeyCommand;
+import org.apache.ignite.internal.table.distributed.command.TransactionalCommand;
 import org.apache.ignite.internal.table.distributed.command.UpsertAllCommand;
 import org.apache.ignite.internal.table.distributed.command.UpsertCommand;
 import org.apache.ignite.internal.table.distributed.command.response.MultiRowsResponse;
@@ -60,15 +57,19 @@ import org.apache.ignite.internal.table.distributed.command.response.SingleRowRe
 import org.apache.ignite.internal.table.distributed.command.scan.ScanCloseCommand;
 import org.apache.ignite.internal.table.distributed.command.scan.ScanInitCommand;
 import org.apache.ignite.internal.table.distributed.command.scan.ScanRetrieveBatchCommand;
+import org.apache.ignite.internal.table.distributed.storage.VersionedRowStore;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.TxState;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.lang.LoggerMessageHelper;
 import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupListener;
+import org.apache.ignite.tx.TransactionException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.TestOnly;
 
@@ -76,19 +77,28 @@ import org.jetbrains.annotations.TestOnly;
  * Partition command handler.
  */
 public class PartitionListener implements RaftGroupListener {
-    /** Partition storage. */
-    private final PartitionStorage storage;
+    /** Lock id. */
+    private final IgniteUuid lockId;
+
+    /** The versioned storage. */
+    private final VersionedRowStore storage;
 
     /** Cursors map. */
     private final Map<IgniteUuid, CursorMeta> cursors;
 
+    /** Transaction manager. */
+    private final TxManager txManager;
+
     /**
-     * Constructor.
+     * The constructor.
      *
-     * @param partitionStorage Storage.
+     * @param lockId Lock id.
+     * @param store  The storage.
      */
-    public PartitionListener(PartitionStorage partitionStorage) {
-        this.storage = partitionStorage;
+    public PartitionListener(IgniteUuid lockId, VersionedRowStore store) {
+        this.lockId = lockId;
+        this.storage = store;
+        this.txManager = store.txManager();
         this.cursors = new ConcurrentHashMap<>();
     }
 
@@ -96,9 +106,15 @@ public class PartitionListener implements RaftGroupListener {
     @Override
     public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
         iterator.forEachRemaining((CommandClosure<? extends ReadCommand> clo) -> {
-            if (clo.command() instanceof GetCommand) {
+            Command command = clo.command();
+
+            if (!tryEnlistIntoTransaction(command, clo)) {
+                return;
+            }
+
+            if (command instanceof GetCommand) {
                 handleGetCommand((CommandClosure<GetCommand>) clo);
-            } else if (clo.command() instanceof GetAllCommand) {
+            } else if (command instanceof GetAllCommand) {
                 handleGetAllCommand((CommandClosure<GetAllCommand>) clo);
             } else {
                 assert false : "Command was not found [cmd=" + clo.command() + ']';
@@ -112,6 +128,10 @@ public class PartitionListener implements RaftGroupListener {
         iterator.forEachRemaining((CommandClosure<? extends WriteCommand> clo) -> {
             Command command = clo.command();
 
+            if (!tryEnlistIntoTransaction(command, clo)) {
+                return;
+            }
+
             if (command instanceof InsertCommand) {
                 handleInsertCommand((CommandClosure<InsertCommand>) clo);
             } else if (command instanceof DeleteCommand) {
@@ -144,6 +164,8 @@ public class PartitionListener implements RaftGroupListener {
                 handleScanRetrieveBatchCommand((CommandClosure<ScanRetrieveBatchCommand>) clo);
             } else if (command instanceof ScanCloseCommand) {
                 handleScanCloseCommand((CommandClosure<ScanCloseCommand>) clo);
+            } else if (command instanceof FinishTxCommand) {
+                handleFinishTxCommand((CommandClosure<FinishTxCommand>) clo);
             } else {
                 assert false : "Command was not found [cmd=" + command + ']';
             }
@@ -151,18 +173,37 @@ public class PartitionListener implements RaftGroupListener {
     }
 
     /**
+     * Attempts to enlist a command into a transaction.
+     *
+     * @param command The command.
+     * @param clo     The closure.
+     * @return {@code true} if a command is compatible with a transaction state or a command is not transactional.
+     */
+    private boolean tryEnlistIntoTransaction(Command command, CommandClosure<?> clo) {
+        if (command instanceof TransactionalCommand) {
+            Timestamp ts = ((TransactionalCommand) command).getTimestamp();
+
+            TxState state = txManager.getOrCreateTransaction(ts);
+
+            if (state != null && state != TxState.PENDING) {
+                clo.result(new TransactionException(format("Failed to enlist a key into a transaction, state={}", state)));
+
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * Handler for the {@link GetCommand}.
      *
      * @param clo Command closure.
      */
     private void handleGetCommand(CommandClosure<GetCommand> clo) {
-        BinaryRow keyRow = clo.command().getKeyRow();
+        GetCommand cmd = clo.command();
 
-        DataRow readValue = storage.read(new BinarySearchRow(keyRow));
-
-        BinaryRow responseRow = readValue == null ? null : new ByteBufferRow(readValue.valueBytes());
-
-        clo.result(new SingleRowResponse(responseRow));
+        clo.result(new SingleRowResponse(storage.get(cmd.getRow(), cmd.getTimestamp())));
     }
 
     /**
@@ -171,19 +212,14 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleGetAllCommand(CommandClosure<GetAllCommand> clo) {
-        Set<BinaryRow> keyRows = clo.command().getKeyRows();
-
-        assert keyRows != null && !keyRows.isEmpty();
+        GetAllCommand cmd = clo.command();
 
-        List<SearchRow> keys = keyRows.stream()
-                .map(BinarySearchRow::new)
-                .collect(Collectors.toList());
+        Collection<BinaryRow> keyRows = cmd.getRows();
 
-        List<BinaryRow> res = storage.readAll(keys).stream()
-                .map(read -> new ByteBufferRow(read.valueBytes()))
-                .collect(Collectors.toList());
+        assert keyRows != null && !keyRows.isEmpty();
 
-        clo.result(new MultiRowsResponse(res));
+        // TODO asch IGNITE-15934 all reads are sequential, can be parallelized ?
+        clo.result(new MultiRowsResponse(storage.getAll(keyRows, cmd.getTimestamp())));
     }
 
     /**
@@ -192,17 +228,9 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleInsertCommand(CommandClosure<InsertCommand> clo) {
-        BinaryRow row = clo.command().getRow();
-
-        assert row.hasValue() : "Insert command should have a value.";
+        InsertCommand cmd = clo.command();
 
-        DataRow newRow = extractAndWrapKeyValue(row);
-
-        var writeIfAbsent = new InsertInvokeClosure(newRow);
-
-        storage.invoke(newRow, writeIfAbsent);
-
-        clo.result(writeIfAbsent.result());
+        clo.result(storage.insert(cmd.getRow(), cmd.getTimestamp()));
     }
 
     /**
@@ -211,15 +239,9 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleDeleteCommand(CommandClosure<DeleteCommand> clo) {
-        BinaryRow keyRow = clo.command().getKeyRow();
-
-        SearchRow newRow = new BinarySearchRow(keyRow);
+        DeleteCommand cmd = clo.command();
 
-        var getAndRemoveClosure = new GetAndRemoveInvokeClosure();
-
-        storage.invoke(newRow, getAndRemoveClosure);
-
-        clo.result(getAndRemoveClosure.result());
+        clo.result(storage.delete(cmd.getRow(), cmd.getTimestamp()));
     }
 
     /**
@@ -228,14 +250,9 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleReplaceCommand(CommandClosure<ReplaceCommand> clo) {
-        DataRow expected = extractAndWrapKeyValue(clo.command().getOldRow());
-        DataRow newRow = extractAndWrapKeyValue(clo.command().getRow());
-
-        var replaceClosure = new ReplaceExactInvokeClosure(expected, newRow);
+        ReplaceCommand cmd = clo.command();
 
-        storage.invoke(expected, replaceClosure);
-
-        clo.result(replaceClosure.result());
+        clo.result(storage.replace(cmd.getOldRow(), cmd.getRow(), cmd.getTimestamp()));
     }
 
     /**
@@ -244,11 +261,9 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleUpsertCommand(CommandClosure<UpsertCommand> clo) {
-        BinaryRow row = clo.command().getRow();
-
-        assert row.hasValue() : "Upsert command should have a value.";
+        UpsertCommand cmd = clo.command();
 
-        storage.write(extractAndWrapKeyValue(row));
+        storage.upsert(cmd.getRow(), cmd.getTimestamp());
 
         clo.result(null);
     }
@@ -259,19 +274,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleInsertAllCommand(CommandClosure<InsertAllCommand> clo) {
-        Set<BinaryRow> rows = clo.command().getRows();
-
-        assert rows != null && !rows.isEmpty();
+        InsertAllCommand cmd = clo.command();
 
-        List<DataRow> keyValues = rows.stream()
-                .map(PartitionListener::extractAndWrapKeyValue)
-                .collect(Collectors.toList());
+        Collection<BinaryRow> rows = cmd.getRows();
 
-        List<BinaryRow> res = storage.insertAll(keyValues).stream()
-                .map(skipped -> new ByteBufferRow(skipped.valueBytes()))
-                .collect(Collectors.toList());
+        assert rows != null && !rows.isEmpty();
 
-        clo.result(new MultiRowsResponse(res));
+        clo.result(new MultiRowsResponse(storage.insertAll(rows, cmd.getTimestamp())));
     }
 
     /**
@@ -280,15 +289,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleUpsertAllCommand(CommandClosure<UpsertAllCommand> clo) {
-        Set<BinaryRow> rows = clo.command().getRows();
+        UpsertAllCommand cmd = clo.command();
 
-        assert rows != null && !rows.isEmpty();
+        Collection<BinaryRow> rows = cmd.getRows();
 
-        List<DataRow> keyValues = rows.stream()
-                .map(PartitionListener::extractAndWrapKeyValue)
-                .collect(Collectors.toList());
+        assert rows != null && !rows.isEmpty();
 
-        storage.writeAll(keyValues);
+        storage.upsertAll(rows, cmd.getTimestamp());
 
         clo.result(null);
     }
@@ -299,19 +306,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleDeleteAllCommand(CommandClosure<DeleteAllCommand> clo) {
-        Set<BinaryRow> rows = clo.command().getRows();
-
-        assert rows != null && !rows.isEmpty();
+        DeleteAllCommand cmd = clo.command();
 
-        List<SearchRow> keys = rows.stream()
-                .map(BinarySearchRow::new)
-                .collect(Collectors.toList());
+        Collection<BinaryRow> rows = cmd.getRows();
 
-        List<BinaryRow> res = storage.removeAll(keys).stream()
-                .map(skipped -> ((BinarySearchRow) skipped).sourceRow)
-                .collect(Collectors.toList());
+        assert rows != null && !rows.isEmpty();
 
-        clo.result(new MultiRowsResponse(res));
+        clo.result(new MultiRowsResponse(storage.deleteAll(rows, cmd.getTimestamp())));
     }
 
     /**
@@ -320,17 +321,14 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleDeleteExactCommand(CommandClosure<DeleteExactCommand> clo) {
-        BinaryRow row = clo.command().getRow();
-
-        assert row != null && row.hasValue();
+        DeleteExactCommand cmd = clo.command();
 
-        DataRow keyValue = extractAndWrapKeyValue(row);
+        BinaryRow row = cmd.getRow();
 
-        var deleteExact = new DeleteExactInvokeClosure(keyValue);
-
-        storage.invoke(keyValue, deleteExact);
+        assert row != null;
+        assert row.hasValue();
 
-        clo.result(deleteExact.result());
+        clo.result(storage.deleteExact(row, cmd.getTimestamp()));
     }
 
     /**
@@ -339,19 +337,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleDeleteExactAllCommand(CommandClosure<DeleteExactAllCommand> clo) {
-        Set<BinaryRow> rows = clo.command().getRows();
-
-        assert rows != null && !rows.isEmpty();
+        DeleteExactAllCommand cmd = clo.command();
 
-        List<DataRow> keyValues = rows.stream()
-                .map(PartitionListener::extractAndWrapKeyValue)
-                .collect(Collectors.toList());
+        Collection<BinaryRow> rows = cmd.getRows();
 
-        List<BinaryRow> res = storage.removeAllExact(keyValues).stream()
-                .map(skipped -> new ByteBufferRow(skipped.valueBytes()))
-                .collect(Collectors.toList());
+        assert rows != null && !rows.isEmpty();
 
-        clo.result(new MultiRowsResponse(res));
+        clo.result(new MultiRowsResponse(storage.deleteAllExact(rows, cmd.getTimestamp())));
     }
 
     /**
@@ -360,17 +352,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleReplaceIfExistsCommand(CommandClosure<ReplaceIfExistCommand> clo) {
-        BinaryRow row = clo.command().getRow();
+        ReplaceIfExistCommand cmd = clo.command();
 
-        assert row != null;
+        BinaryRow row = cmd.getRow();
 
-        DataRow keyValue = extractAndWrapKeyValue(row);
-
-        var replaceIfExists = new GetAndReplaceInvokeClosure(keyValue, true);
-
-        storage.invoke(keyValue, replaceIfExists);
+        assert row != null;
 
-        clo.result(replaceIfExists.result());
+        clo.result(storage.replace(row, cmd.getTimestamp()));
     }
 
     /**
@@ -379,20 +367,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleGetAndDeleteCommand(CommandClosure<GetAndDeleteCommand> clo) {
-        BinaryRow row = clo.command().getKeyRow();
-
-        assert row != null;
-
-        SearchRow keyRow = new BinarySearchRow(row);
-
-        var getAndRemoveClosure = new GetAndRemoveInvokeClosure();
+        GetAndDeleteCommand cmd = clo.command();
 
-        storage.invoke(keyRow, getAndRemoveClosure);
+        BinaryRow row = cmd.getRow();
 
-        BinaryRow removedRow = getAndRemoveClosure.result()
-                ? new ByteBufferRow(getAndRemoveClosure.oldRow().valueBytes()) : null;
+        assert row != null;
 
-        clo.result(new SingleRowResponse(removedRow));
+        clo.result(new SingleRowResponse(storage.getAndDelete(row, cmd.getTimestamp())));
     }
 
     /**
@@ -401,21 +382,13 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleGetAndReplaceCommand(CommandClosure<GetAndReplaceCommand> clo) {
-        BinaryRow row = clo.command().getRow();
-
-        assert row != null && row.hasValue();
-
-        DataRow keyValue = extractAndWrapKeyValue(row);
+        GetAndReplaceCommand cmd = clo.command();
 
-        var getAndReplace = new GetAndReplaceInvokeClosure(keyValue, true);
+        BinaryRow row = cmd.getRow();
 
-        storage.invoke(keyValue, getAndReplace);
-
-        DataRow oldRow = getAndReplace.oldRow();
-
-        BinaryRow res = oldRow == null ? null : new ByteBufferRow(oldRow.valueBytes());
+        assert row != null && row.hasValue();
 
-        clo.result(new SingleRowResponse(res));
+        clo.result(new SingleRowResponse(storage.getAndReplace(row, cmd.getTimestamp())));
     }
 
     /**
@@ -424,21 +397,27 @@ public class PartitionListener implements RaftGroupListener {
      * @param clo Command closure.
      */
     private void handleGetAndUpsertCommand(CommandClosure<GetAndUpsertCommand> clo) {
-        BinaryRow row = clo.command().getKeyRow();
+        GetAndUpsertCommand cmd = clo.command();
 
-        assert row != null && row.hasValue();
-
-        DataRow keyValue = extractAndWrapKeyValue(row);
+        BinaryRow row = cmd.getRow();
 
-        var getAndReplace = new GetAndReplaceInvokeClosure(keyValue, false);
+        assert row != null && row.hasValue();
 
-        storage.invoke(keyValue, getAndReplace);
+        clo.result(new SingleRowResponse(storage.getAndUpsert(row, cmd.getTimestamp())));
+    }
 
-        DataRow oldRow = getAndReplace.oldRow();
+    /**
+     * Handler for the {@link FinishTxCommand}.
+     *
+     * @param clo Command closure.
+     */
+    private void handleFinishTxCommand(CommandClosure<FinishTxCommand> clo) {
+        FinishTxCommand cmd = clo.command();
 
-        BinaryRow response = oldRow == null ? null : new ByteBufferRow(oldRow.valueBytes());
+        Timestamp ts = cmd.timestamp();
+        boolean commit = cmd.finish();
 
-        clo.result(new SingleRowResponse(response));
+        clo.result(txManager.changeState(ts, TxState.PENDING, commit ? TxState.COMMITED : TxState.ABORTED));
     }
 
     /**
@@ -452,7 +431,7 @@ public class PartitionListener implements RaftGroupListener {
         IgniteUuid cursorId = rangeCmd.scanId();
 
         try {
-            Cursor<DataRow> cursor = storage.scan(key -> true);
+            Cursor<BinaryRow> cursor = storage.scan(key -> true);
 
             cursors.put(
                     cursorId,
@@ -477,7 +456,7 @@ public class PartitionListener implements RaftGroupListener {
         CursorMeta cursorDesc = cursors.get(clo.command().scanId());
 
         if (cursorDesc == null) {
-            clo.result(new NoSuchElementException(LoggerMessageHelper.format(
+            clo.result(new NoSuchElementException(format(
                     "Cursor with id={} is not found on server side.", clo.command().scanId())));
 
             return;
@@ -487,7 +466,7 @@ public class PartitionListener implements RaftGroupListener {
 
         try {
             for (int i = 0; i < clo.command().itemsToRetrieveCount() && cursorDesc.cursor().hasNext(); i++) {
-                res.add(new ByteBufferRow(cursorDesc.cursor().next().valueBytes()));
+                res.add(cursorDesc.cursor().next());
             }
         } catch (NoSuchElementException e) {
             clo.result(e);
@@ -531,6 +510,7 @@ public class PartitionListener implements RaftGroupListener {
     @Override
     public boolean onSnapshotLoad(Path path) {
         storage.restoreSnapshot(path);
+
         return true;
     }
 
@@ -544,6 +524,35 @@ public class PartitionListener implements RaftGroupListener {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<Void> onBeforeApply(Command command) {
+        if (command instanceof SingleKeyCommand) {
+            SingleKeyCommand cmd0 = (SingleKeyCommand) command;
+
+            return cmd0 instanceof ReadCommand ? txManager.readLock(lockId, cmd0.getRow().keySlice(), cmd0.getTimestamp()) :
+                    txManager.writeLock(lockId, cmd0.getRow().keySlice(), cmd0.getTimestamp());
+        } else if (command instanceof MultiKeyCommand) {
+            MultiKeyCommand cmd0 = (MultiKeyCommand) command;
+
+            Collection<BinaryRow> rows = cmd0.getRows();
+
+            CompletableFuture<Void>[] futs = new CompletableFuture[rows.size()];
+
+            int i = 0;
+            boolean read = cmd0 instanceof ReadCommand;
+
+            for (BinaryRow row : rows) {
+                futs[i++] = read ? txManager.readLock(lockId, row.keySlice(), cmd0.getTimestamp()) :
+                        txManager.writeLock(lockId, row.keySlice(), cmd0.getTimestamp());
+            }
+
+            return CompletableFuture.allOf(futs);
+        }
+
+        return null;
+    }
+
     /**
      * Extracts a key and a value from the {@link BinaryRow} and wraps it in a {@link DataRow}.
      *
@@ -560,54 +569,19 @@ public class PartitionListener implements RaftGroupListener {
     }
 
     /**
-     * Adapter that converts a {@link BinaryRow} into a {@link SearchRow}.
-     */
-    private static class BinarySearchRow implements SearchRow {
-        /** Search key. */
-        private final byte[] keyBytes;
-
-        /** Source row. */
-        private final BinaryRow sourceRow;
-
-        /**
-         * Constructor.
-         *
-         * @param row Row to search for.
-         */
-        BinarySearchRow(BinaryRow row) {
-            sourceRow = row;
-            keyBytes = new byte[row.keySlice().capacity()];
-
-            row.keySlice().get(keyBytes);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public byte @NotNull [] keyBytes() {
-            return keyBytes;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public @NotNull ByteBuffer key() {
-            return ByteBuffer.wrap(keyBytes);
-        }
-    }
-
-    /**
      * Returns underlying storage.
      */
     @TestOnly
-    public PartitionStorage getStorage() {
+    public VersionedRowStore getStorage() {
         return storage;
     }
 
     /**
      * Cursor meta information: origin node id and type.
      */
-    private class CursorMeta {
+    private static class CursorMeta {
         /** Cursor. */
-        private final Cursor<DataRow> cursor;
+        private final Cursor<BinaryRow> cursor;
 
         /** Id of the node that creates cursor. */
         private final String requesterNodeId;
@@ -615,27 +589,20 @@ public class PartitionListener implements RaftGroupListener {
         /**
          * The constructor.
          *
-         * @param cursor          Cursor.
+         * @param cursor          The cursor.
          * @param requesterNodeId Id of the node that creates cursor.
          */
-        CursorMeta(
-                Cursor<DataRow> cursor,
-                String requesterNodeId
-        ) {
+        CursorMeta(Cursor<BinaryRow> cursor, String requesterNodeId) {
             this.cursor = cursor;
             this.requesterNodeId = requesterNodeId;
         }
 
-        /**
-         * Returns cursor.
-         */
-        public Cursor<DataRow> cursor() {
+        /** Returns cursor. */
+        public Cursor<BinaryRow> cursor() {
             return cursor;
         }
 
-        /**
-         * Returns id of the node that creates cursor.
-         */
+        /** Returns id of the node that creates cursor. */
         public String requesterNodeId() {
             return requesterNodeId;
         }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
index 72352a5..fb60275 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
@@ -17,20 +17,22 @@
 
 package org.apache.ignite.internal.table.distributed.storage;
 
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Flow.Publisher;
 import java.util.concurrent.Flow.Subscriber;
 import java.util.concurrent.Flow.Subscription;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -56,16 +58,20 @@ import org.apache.ignite.internal.table.distributed.command.response.SingleRowRe
 import org.apache.ignite.internal.table.distributed.command.scan.ScanCloseCommand;
 import org.apache.ignite.internal.table.distributed.command.scan.ScanInitCommand;
 import org.apache.ignite.internal.table.distributed.command.scan.ScanRetrieveBatchCommand;
+import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.TxManager;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.IgniteUuidGenerator;
 import org.apache.ignite.lang.LoggerMessageHelper;
 import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.service.RaftGroupService;
-import org.apache.ignite.tx.Transaction;
+import org.apache.ignite.tx.TransactionException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
+import org.jetbrains.annotations.TestOnly;
 
 /**
  * Storage of table rows.
@@ -79,7 +85,7 @@ public class InternalTableImpl implements InternalTable {
 
     //TODO: IGNITE-15443 Use IntMap structure instead of HashMap.
     /** Partition map. */
-    private final Map<Integer, RaftGroupService> partitionMap;
+    protected final Map<Integer, RaftGroupService> partitionMap;
 
     /** Partitions. */
     private final int partitions;
@@ -93,16 +99,21 @@ public class InternalTableImpl implements InternalTable {
     /** Resolver that resolves a network address to node id. */
     private final Function<NetworkAddress, String> netAddrResolver;
 
+    /** Transactional manager. */
+    private final TxManager txManager;
+
     /** Storage for table data. */
     private final TableStorage tableStorage;
 
     /**
      * Constructor.
      *
-     * @param tableName  Table name.
-     * @param tableId    Table id.
-     * @param partMap    Map partition id to raft group.
+     * @param tableName Table name.
+     * @param tableId Table id.
+     * @param partMap Map partition id to raft group.
      * @param partitions Partitions.
+     * @param txManager Transaction manager.
+     * @param tableStorage Table storage.
      */
     public InternalTableImpl(
             String tableName,
@@ -110,6 +121,7 @@ public class InternalTableImpl implements InternalTable {
             Map<Integer, RaftGroupService> partMap,
             int partitions,
             Function<NetworkAddress, String> netAddrResolver,
+            TxManager txManager,
             TableStorage tableStorage
     ) {
         this.tableName = tableName;
@@ -117,6 +129,7 @@ public class InternalTableImpl implements InternalTable {
         this.partitionMap = partMap;
         this.partitions = partitions;
         this.netAddrResolver = netAddrResolver;
+        this.txManager = txManager;
         this.tableStorage = tableStorage;
     }
 
@@ -144,167 +157,212 @@ public class InternalTableImpl implements InternalTable {
         return tableName;
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<BinaryRow> get(BinaryRow keyRow, Transaction tx) {
-        return partitionMap.get(partId(keyRow)).<SingleRowResponse>run(new GetCommand(keyRow))
-                .thenApply(SingleRowResponse::getValue);
-    }
+    /**
+     * Enlists multiple rows into a transaction.
+     *
+     * @param keyRows Rows.
+     * @param tx The transaction.
+     * @param op Command factory.
+     * @param reducer The reducer.
+     * @param <R> Reducer's input.
+     * @param <T> Reducer's output.
+     * @return The future.
+     */
+    private <R, T> CompletableFuture<T> enlistInTx(
+            Collection<BinaryRow> keyRows,
+            InternalTransaction tx,
+            BiFunction<Collection<BinaryRow>, InternalTransaction, Command> op,
+            Function<CompletableFuture<R>[], CompletableFuture<T>> reducer
+    ) {
+        if (tx == null) {
+            try {
+                tx = txManager.tx();
+            } catch (TransactionException e) {
+                return failedFuture(e);
+            }
+        }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows, Transaction tx) {
-        Map<Integer, Set<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(keyRows);
+        final boolean implicit = tx == null;
+
+        final InternalTransaction tx0 = implicit ? txManager.begin() : tx;
+
+        Map<Integer, List<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(keyRows);
 
-        CompletableFuture<MultiRowsResponse>[] futures = new CompletableFuture[keyRowsByPartition.size()];
+        CompletableFuture<R>[] futures = new CompletableFuture[keyRowsByPartition.size()];
 
         int batchNum = 0;
 
-        for (Map.Entry<Integer, Set<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
-            futures[batchNum] = partitionMap.get(partToRows.getKey()).run(new GetAllCommand(partToRows.getValue()));
+        for (Map.Entry<Integer, List<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
+            CompletableFuture<RaftGroupService> fut = enlist(partToRows.getKey(), tx0);
 
-            batchNum++;
+            futures[batchNum++] = fut.thenCompose(svc -> svc.run(op.apply(partToRows.getValue(), tx0)));
         }
 
-        return collectMultiRowsResponses(futures);
-    }
+        CompletableFuture<T> fut = reducer.apply(futures);
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> upsert(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).run(new UpsertCommand(row));
+        return postEnlist(fut, implicit, tx0);
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, Transaction tx) {
-        Map<Integer, Set<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(rows);
+    /**
+     * Enlists a single row into a transaction.
+     *
+     * @param row The row.
+     * @param tx The transaction.
+     * @param op Command factory.
+     * @param trans Transform closure.
+     * @param <R> Transform input.
+     * @param <T> Transform output.
+     * @return The future.
+     */
+    private <R, T> CompletableFuture<T> enlistInTx(
+            BinaryRow row,
+            InternalTransaction tx,
+            Function<InternalTransaction, Command> op,
+            Function<R, T> trans
+    ) {
+        if (tx == null) {
+            try {
+                tx = txManager.tx();
+            } catch (TransactionException e) {
+                return failedFuture(e);
+            }
+        }
 
-        CompletableFuture<Void>[] futures = new CompletableFuture[keyRowsByPartition.size()];
+        final boolean implicit = tx == null;
 
-        int batchNum = 0;
+        final InternalTransaction tx0 = implicit ? txManager.begin() : tx;
 
-        for (Map.Entry<Integer, Set<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
-            futures[batchNum] = partitionMap.get(partToRows.getKey()).run(new UpsertAllCommand(partToRows.getValue()));
+        int partId = partId(row);
 
-            batchNum++;
-        }
+        CompletableFuture<T> fut = enlist(partId, tx0).thenCompose(svc -> svc.<R>run(op.apply(tx0)).thenApply(trans::apply));
+
+        return postEnlist(fut, implicit, tx0);
+    }
+
+    /**
+     * Performs post enlist operation.
+     *
+     * @param fut The future.
+     * @param implicit {@code true} for implicit tx.
+     * @param tx0 The transaction.
+     * @param <T> Operation return type.
+     * @return The future.
+     */
+    private <T> CompletableFuture<T> postEnlist(CompletableFuture<T> fut, boolean implicit, InternalTransaction tx0) {
+        return fut.handle(new BiFunction<T, Throwable, CompletableFuture<T>>() {
+            @Override
+            public CompletableFuture<T> apply(T r, Throwable e) {
+                if (e != null) {
+                    return tx0.rollbackAsync().handle((ignored, err) -> {
+                        if (err != null) {
+                            e.addSuppressed(err);
+                        }
 
-        return CompletableFuture.allOf(futures);
+                        throw (RuntimeException) e;
+                    }); // Preserve failed state.
+                } else {
+                    return implicit ? tx0.commitAsync().thenApply(ignored -> r) : completedFuture(r);
+                }
+            }
+        }).thenCompose(x -> x);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).<SingleRowResponse>run(new GetAndUpsertCommand(row))
-                .thenApply(SingleRowResponse::getValue);
+    public CompletableFuture<BinaryRow> get(BinaryRow keyRow, InternalTransaction tx) {
+        return enlistInTx(keyRow, tx, tx0 -> new GetCommand(keyRow, tx0.timestamp()), SingleRowResponse::getValue);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> insert(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).run(new InsertCommand(row));
+    public CompletableFuture<Collection<BinaryRow>> getAll(Collection<BinaryRow> keyRows, InternalTransaction tx) {
+        return enlistInTx(keyRows, tx, (rows0, tx0) -> new GetAllCommand(rows0, tx0.timestamp()), this::collectMultiRowsResponses);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, Transaction tx) {
-        Map<Integer, Set<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(rows);
-
-        CompletableFuture<MultiRowsResponse>[] futures = new CompletableFuture[keyRowsByPartition.size()];
-
-        int batchNum = 0;
-
-        for (Map.Entry<Integer, Set<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
-            futures[batchNum] = partitionMap.get(partToRows.getKey()).run(new InsertAllCommand(partToRows.getValue()));
-
-            batchNum++;
-        }
-
-        return collectMultiRowsResponses(futures);
+    public CompletableFuture<Void> upsert(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new UpsertCommand(row, tx0.timestamp()), ignored -> null);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> replace(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).<Boolean>run(new ReplaceIfExistCommand(row));
+    public CompletableFuture<Void> upsertAll(Collection<BinaryRow> rows, InternalTransaction tx) {
+        return enlistInTx(rows, tx, (rows0, tx0) -> new UpsertAllCommand(rows0, tx0.timestamp()), CompletableFuture::allOf);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow,
-            Transaction tx) {
-        return partitionMap.get(partId(oldRow)).run(new ReplaceCommand(oldRow, newRow));
+    public CompletableFuture<BinaryRow> getAndUpsert(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new GetAndUpsertCommand(row, tx0.timestamp()), SingleRowResponse::getValue);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndReplace(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).<SingleRowResponse>run(new GetAndReplaceCommand(row))
-                .thenApply(SingleRowResponse::getValue);
+    public CompletableFuture<Boolean> insert(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new InsertCommand(row, tx0.timestamp()), r -> (Boolean) r);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> delete(BinaryRow keyRow, Transaction tx) {
-        return partitionMap.get(partId(keyRow)).run(new DeleteCommand(keyRow));
+    public CompletableFuture<Collection<BinaryRow>> insertAll(Collection<BinaryRow> rows, InternalTransaction tx) {
+        return enlistInTx(rows, tx, (rows0, tx0) -> new InsertAllCommand(rows0, tx0.timestamp()), this::collectMultiRowsResponses);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, Transaction tx) {
-        return partitionMap.get(partId(oldRow)).<Boolean>run(new DeleteExactCommand(oldRow));
+    public CompletableFuture<Boolean> replace(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new ReplaceIfExistCommand(row, tx0.timestamp()), r -> (Boolean) r);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<BinaryRow> getAndDelete(BinaryRow row, Transaction tx) {
-        return partitionMap.get(partId(row)).<SingleRowResponse>run(new GetAndDeleteCommand(row))
-                .thenApply(SingleRowResponse::getValue);
+    public CompletableFuture<Boolean> replace(BinaryRow oldRow, BinaryRow newRow, InternalTransaction tx) {
+        return enlistInTx(oldRow, tx, tx0 -> new ReplaceCommand(oldRow, newRow, tx0.timestamp()), r -> (Boolean) r);
     }
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, Transaction tx) {
-        Map<Integer, Set<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(rows);
-
-        CompletableFuture<MultiRowsResponse>[] futures = new CompletableFuture[keyRowsByPartition.size()];
+    public CompletableFuture<BinaryRow> getAndReplace(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new GetAndReplaceCommand(row, tx0.timestamp()), SingleRowResponse::getValue);
+    }
 
-        int batchNum = 0;
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<Boolean> delete(BinaryRow keyRow, InternalTransaction tx) {
+        return enlistInTx(keyRow, tx, tx0 -> new DeleteCommand(keyRow, tx0.timestamp()), r -> (Boolean) r);
+    }
 
-        for (Map.Entry<Integer, Set<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
-            futures[batchNum] = partitionMap.get(partToRows.getKey()).run(new DeleteAllCommand(partToRows.getValue()));
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<Boolean> deleteExact(BinaryRow oldRow, InternalTransaction tx) {
+        return enlistInTx(oldRow, tx, tx0 -> new DeleteExactCommand(oldRow, tx0.timestamp()), r -> (Boolean) r);
+    }
 
-            batchNum++;
-        }
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<BinaryRow> getAndDelete(BinaryRow row, InternalTransaction tx) {
+        return enlistInTx(row, tx, tx0 -> new GetAndDeleteCommand(row, tx0.timestamp()), SingleRowResponse::getValue);
+    }
 
-        return collectMultiRowsResponses(futures);
+    /** {@inheritDoc} */
+    @Override
+    public CompletableFuture<Collection<BinaryRow>> deleteAll(Collection<BinaryRow> rows, InternalTransaction tx) {
+        return enlistInTx(rows, tx, (rows0, tx0) -> new DeleteAllCommand(rows0, tx0.timestamp()), this::collectMultiRowsResponses);
     }
 
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Collection<BinaryRow>> deleteAllExact(
             Collection<BinaryRow> rows,
-            Transaction tx
+            InternalTransaction tx
     ) {
-        Map<Integer, Set<BinaryRow>> keyRowsByPartition = mapRowsToPartitions(rows);
-
-        CompletableFuture<MultiRowsResponse>[] futures = new CompletableFuture[keyRowsByPartition.size()];
-
-        int batchNum = 0;
-
-        for (Map.Entry<Integer, Set<BinaryRow>> partToRows : keyRowsByPartition.entrySet()) {
-            futures[batchNum] = partitionMap.get(partToRows.getKey()).run(new DeleteExactAllCommand(partToRows.getValue()));
-
-            batchNum++;
-        }
-
-        return collectMultiRowsResponses(futures);
+        return enlistInTx(rows, tx, (rows0, tx0) -> new DeleteExactAllCommand(rows0, tx0.timestamp()), this::collectMultiRowsResponses);
     }
 
     /** {@inheritDoc} */
     @Override
-    public @NotNull Publisher<BinaryRow> scan(int p, @Nullable Transaction tx) {
+    public @NotNull Publisher<BinaryRow> scan(int p, @Nullable InternalTransaction tx) {
         if (p < 0 || p >= partitions) {
             throw new IllegalArgumentException(
                     LoggerMessageHelper.format(
@@ -325,12 +383,12 @@ public class InternalTableImpl implements InternalTable {
      * @param rows Rows.
      * @return Partition -%gt; rows mapping.
      */
-    private Map<Integer, Set<BinaryRow>> mapRowsToPartitions(Collection<BinaryRow> rows) {
+    private Map<Integer, List<BinaryRow>> mapRowsToPartitions(Collection<BinaryRow> rows) {
         //TODO: IGNITE-15443 Use IntMap structure instead of HashMap.
-        HashMap<Integer, Set<BinaryRow>> keyRowsByPartition = new HashMap<>();
+        HashMap<Integer, List<BinaryRow>> keyRowsByPartition = new HashMap<>();
 
         for (BinaryRow keyRow : rows) {
-            keyRowsByPartition.computeIfAbsent(partId(keyRow), k -> new HashSet<>()).add(keyRow);
+            keyRowsByPartition.computeIfAbsent(partId(keyRow), k -> new ArrayList<>()).add(keyRow);
         }
 
         return keyRowsByPartition;
@@ -362,6 +420,13 @@ public class InternalTableImpl implements InternalTable {
         CompletableFuture.allOf(futs.toArray(CompletableFuture[]::new)).join();
     }
 
+    /** {@inheritDoc} */
+    @TestOnly
+    @Override
+    public int partition(BinaryRow keyRow) {
+        return partId(keyRow);
+    }
+
     /**
      * Get partition id by key row.
      *
@@ -375,19 +440,30 @@ public class InternalTableImpl implements InternalTable {
     }
 
     /**
-     * Collects multirow responses from multiple futures into a single collection.
+     * Returns a transaction manager.
      *
-     * @param futures Futures.
+     * @return Transaction manager.
+     */
+    @TestOnly
+    public TxManager transactionManager() {
+        return txManager;
+    }
+
+    /**
+     * TODO asch keep the same order as for keys Collects multirow responses from multiple futures into a single collection IGNITE-16004.
+     *
+     * @param futs Futures.
      * @return Row collection.
      */
-    private CompletableFuture<Collection<BinaryRow>> collectMultiRowsResponses(
-            CompletableFuture<MultiRowsResponse>[] futures) {
-        return CompletableFuture.allOf(futures)
+    private CompletableFuture<Collection<BinaryRow>> collectMultiRowsResponses(CompletableFuture<?>[] futs) {
+        return CompletableFuture.allOf(futs)
                 .thenApply(response -> {
-                    List<BinaryRow> list = new ArrayList<>(futures.length);
+                    List<BinaryRow> list = new ArrayList<>(futs.length);
+
+                    for (CompletableFuture<?> future : futs) {
+                        MultiRowsResponse ret = (MultiRowsResponse) future.join();
 
-                    for (CompletableFuture<MultiRowsResponse> future : futures) {
-                        Collection<BinaryRow> values = future.join().getValues();
+                        List<BinaryRow> values = ret.getValues();
 
                         if (values != null) {
                             list.addAll(values);
@@ -401,7 +477,7 @@ public class InternalTableImpl implements InternalTable {
     /**
      * Updates internal table raft group service for given partition.
      *
-     * @param p          Partition.
+     * @param p Partition.
      * @param raftGrpSvc Raft group service.
      */
     public void updateInternalTableRaftGroupService(int p, RaftGroupService raftGrpSvc) {
@@ -412,8 +488,27 @@ public class InternalTableImpl implements InternalTable {
         }
     }
 
-    /** Partition scan publisher. */
-    private class PartitionScanPublisher implements Publisher<BinaryRow> {
+    /**
+     * Enlists a partition.
+     *
+     * @param partId Partition id.
+     * @param tx     The transaction.
+     * @return The enlist future (then will a leader become known).
+     */
+    protected CompletableFuture<RaftGroupService> enlist(int partId, InternalTransaction tx) {
+        RaftGroupService svc = partitionMap.get(partId);
+
+        CompletableFuture<Void> fut0 = svc.leader() == null ? svc.refreshLeader() : completedFuture(null);
+
+        // TODO asch IGNITE-15091 fixme need to map to the same leaseholder.
+        // TODO asch a leader race is possible when enlisting different keys from the same partition.
+        return fut0.thenAccept(ignored -> tx.enlist(svc)).thenApply(ignored -> svc); // Enlist the leaseholder.
+    }
+
+    /**
+     * Partition scan publisher.
+     */
+    private static class PartitionScanPublisher implements Publisher<BinaryRow> {
         /** {@link Publisher} that relatively notifies about partition rows. */
         private final RaftGroupService raftGrpSvc;
 
@@ -453,10 +548,14 @@ public class InternalTableImpl implements InternalTable {
 
             private final AtomicBoolean canceled;
 
-            /** Scan id to uniquely identify it on server side. */
+            /**
+             * Scan id to uniquely identify it on server side.
+             */
             private final IgniteUuid scanId;
 
-            /** Scan initial operation that created server cursor. */
+            /**
+             * Scan initial operation that created server cursor.
+             */
             private final CompletableFuture<Void> scanInitOp;
 
             /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/VersionedRowStore.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/VersionedRowStore.java
new file mode 100644
index 0000000..f5fe9e0
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/VersionedRowStore.java
@@ -0,0 +1,754 @@
+/*
+ * 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.table.distributed.storage;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletionStage;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.DataRow;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.SearchRow;
+import org.apache.ignite.internal.storage.basic.SimpleDataRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.TxState;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.Pair;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * TODO asch IGNITE-15934 use read only buffers ? replace Pair from ignite-schema
+ * TODO asch IGNITE-15935 can use some sort of a cache on tx coordinator to avoid network IO.
+ * TODO asch IGNITE-15934 invokes on storage not used for now, can it be changed ?
+ */
+public class VersionedRowStore {
+    /** Storage delegate. */
+    private final PartitionStorage storage;
+
+    /** Transaction manager. */
+    private TxManager txManager;
+
+    /**
+     * The constructor.
+     *
+     * @param storage The storage.
+     * @param txManager The TX manager.
+     */
+    public VersionedRowStore(@NotNull PartitionStorage storage, @NotNull TxManager txManager) {
+        this.storage = Objects.requireNonNull(storage);
+        this.txManager = Objects.requireNonNull(txManager);
+    }
+
+    /**
+     * Decodes a storage row to a pair where the first is an actual value (visible to a current transaction) and the second is an old value
+     * used for rollback.
+     *
+     * @param row       The row.
+     * @param timestamp Timestamp timestamp.
+     * @return Actual value.
+     */
+    protected Pair<BinaryRow, BinaryRow> versionedRow(@Nullable DataRow row, Timestamp timestamp) {
+        return resolve(unpack(row), timestamp);
+    }
+
+    /**
+     * Gets a row.
+     *
+     * @param row The search row.
+     * @param ts The timestamp.
+     * @return The result row.
+     */
+    public BinaryRow get(@NotNull BinaryRow row, Timestamp ts) {
+        assert row != null;
+
+        SearchRow key = extractAndWrapKey(row);
+
+        DataRow readValue = storage.read(key);
+
+        return versionedRow(readValue, ts).getFirst();
+    }
+
+    /**
+     * Gets multiple rows.
+     *
+     * @param keyRows Search rows.
+     * @param ts The timestamp.
+     * @return The result rows.
+     */
+    public List<BinaryRow> getAll(Collection<BinaryRow> keyRows, Timestamp ts) {
+        assert keyRows != null && !keyRows.isEmpty();
+
+        List<BinaryRow> res = new ArrayList<>(keyRows.size());
+
+        for (BinaryRow keyRow : keyRows) {
+            res.add(get(keyRow, ts));
+        }
+
+        return res;
+    }
+
+    /**
+     * Upserts a row.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     */
+    public void upsert(@NotNull BinaryRow row, Timestamp ts) {
+        assert row != null;
+
+        SimpleDataRow key = new SimpleDataRow(extractAndWrapKey(row).keyBytes(), null);
+
+        Pair<BinaryRow, BinaryRow> pair = resolve(unpack(storage.read(key)), ts);
+
+        storage.write(pack(key, new Value(row, pair.getSecond(), ts)));
+    }
+
+    /**
+     * Upserts a row and returns previous value.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return Previous row.
+     */
+    @Nullable
+    public BinaryRow getAndUpsert(@NotNull BinaryRow row, Timestamp ts) {
+        assert row != null;
+
+        BinaryRow oldRow = get(row, ts);
+
+        upsert(row, ts);
+
+        return oldRow != null ? oldRow : null;
+    }
+
+    /**
+     * Deletes a row.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return {@code True} if was deleted.
+     */
+    public boolean delete(BinaryRow row, Timestamp ts) {
+        assert row != null;
+
+        SimpleDataRow key = new SimpleDataRow(extractAndWrapKey(row).keyBytes(), null);
+
+        Pair<BinaryRow, BinaryRow> pair = resolve(unpack(storage.read(key)), ts);
+
+        if (pair.getFirst() == null) {
+            return false;
+        }
+
+        // Write a tombstone.
+        storage.write(pack(key, new Value(null, pair.getSecond(), ts)));
+
+        return true;
+    }
+
+    /**
+     * Upserts multiple rows.
+     *
+     * @param rows Search rows.
+     * @param ts The timestamp.
+     */
+    public void upsertAll(Collection<BinaryRow> rows, Timestamp ts) {
+        assert rows != null && !rows.isEmpty();
+
+        for (BinaryRow row : rows) {
+            upsert(row, ts);
+        }
+    }
+
+    /**
+     * Inserts a row.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return {@code true} if was inserted.
+     */
+    public boolean insert(BinaryRow row, Timestamp ts) {
+        assert row != null && row.hasValue() : row;
+        SimpleDataRow key = new SimpleDataRow(extractAndWrapKey(row).keyBytes(), null);
+
+        Pair<BinaryRow, BinaryRow> pair = resolve(unpack(storage.read(key)), ts);
+
+        if (pair.getFirst() != null) {
+            return false;
+        }
+
+        storage.write(pack(key, new Value(row, null, ts)));
+
+        return true;
+    }
+
+    /**
+     * Inserts multiple rows.
+     *
+     * @param rows Rows.
+     * @param ts The timestamp.
+     * @return List of not inserted rows.
+     */
+    public List<BinaryRow> insertAll(Collection<BinaryRow> rows, Timestamp ts) {
+        assert rows != null && !rows.isEmpty();
+
+        List<BinaryRow> inserted = new ArrayList<>(rows.size());
+
+        for (BinaryRow row : rows) {
+            if (!insert(row, ts)) {
+                inserted.add(row);
+            }
+        }
+
+        return inserted;
+    }
+
+    /**
+     * Replaces an existing row.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return {@code True} if was replaced.
+     */
+    public boolean replace(BinaryRow row, Timestamp ts) {
+        assert row != null;
+
+        BinaryRow oldRow = get(row, ts);
+
+        if (oldRow != null) {
+            upsert(row, ts);
+
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Replaces a row by exact match.
+     *
+     * @param oldRow Old row.
+     * @param newRow New row.
+     * @param ts The timestamp.
+     * @return {@code True} if was replaced.
+     */
+    public boolean replace(BinaryRow oldRow, BinaryRow newRow, Timestamp ts) {
+        assert oldRow != null;
+        assert newRow != null;
+
+        BinaryRow oldRow0 = get(oldRow, ts);
+
+        if (oldRow0 != null && equalValues(oldRow0, oldRow)) {
+            upsert(newRow, ts);
+
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Replaces existing row and returns a previous value.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return Replaced row.
+     */
+    public BinaryRow getAndReplace(BinaryRow row, Timestamp ts) {
+        BinaryRow oldRow = get(row, ts);
+
+        if (oldRow != null) {
+            upsert(row, ts);
+
+            return oldRow;
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Deletes a row by exact match.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return {@code True} if was deleted.
+     */
+    public boolean deleteExact(BinaryRow row, Timestamp ts) {
+        assert row != null;
+        assert row.hasValue();
+
+        BinaryRow oldRow = get(row, ts);
+
+        if (oldRow != null && equalValues(oldRow, row)) {
+            delete(oldRow, ts);
+
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Delets a row and returns a previous value.
+     *
+     * @param row The row.
+     * @param ts The timestamp.
+     * @return Deleted row.
+     */
+    public BinaryRow getAndDelete(BinaryRow row, Timestamp ts) {
+        BinaryRow oldRow = get(row, ts);
+
+        if (oldRow != null) {
+            delete(oldRow, ts);
+
+            return oldRow;
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Deletes multiple rows.
+     *
+     * @param keyRows Search rows.
+     * @param ts The timestamp.
+     * @return Not deleted rows.
+     */
+    public List<BinaryRow> deleteAll(Collection<BinaryRow> keyRows, Timestamp ts) {
+        var notDeleted = new ArrayList<BinaryRow>();
+
+        for (BinaryRow keyRow : keyRows) {
+            if (!delete(keyRow, ts)) {
+                notDeleted.add(keyRow);
+            }
+        }
+
+        return notDeleted;
+    }
+
+    /**
+     * Deletes multiple rows by exact match.
+     *
+     * @param rows Search rows.
+     * @param ts The timestamp.
+     * @return Not deleted rows.
+     */
+    public List<BinaryRow> deleteAllExact(Collection<BinaryRow> rows, Timestamp ts) {
+        assert rows != null && !rows.isEmpty();
+
+        var notDeleted = new ArrayList<BinaryRow>(rows.size());
+
+        for (BinaryRow row : rows) {
+            if (!deleteExact(row, ts)) {
+                notDeleted.add(row);
+            }
+        }
+
+        return notDeleted;
+    }
+
+    /**
+     * Tests row values for equality.
+     *
+     * @param row Row.
+     * @return Extracted key.
+     */
+    private boolean equalValues(@NotNull BinaryRow row, @NotNull BinaryRow row2) {
+        if (row.hasValue() ^ row2.hasValue()) {
+            return false;
+        }
+
+        return row.valueSlice().compareTo(row2.valueSlice()) == 0;
+    }
+
+    /**
+     * Closes a storage.
+     *
+     * @throws Exception If failed.
+     */
+    public void close() throws Exception {
+        storage.close();
+    }
+
+    /**
+     * Extracts a key and a value from the {@link BinaryRow} and wraps it in a {@link DataRow}.
+     *
+     * @param row Binary row.
+     * @return Data row.
+     */
+    @NotNull
+    private static DataRow extractAndWrapKeyValue(@NotNull BinaryRow row) {
+        byte[] key = new byte[row.keySlice().capacity()];
+        row.keySlice().get(key);
+
+        return new SimpleDataRow(key, row.hasValue() ? row.bytes() : null);
+    }
+
+    /**
+     * Extracts a key from the {@link BinaryRow} and wraps it in a {@link SearchRow}.
+     *
+     * @param row Binary row.
+     * @return Search row.
+     */
+    @NotNull
+    private static SearchRow extractAndWrapKey(@NotNull BinaryRow row) {
+        // TODO asch IGNITE-15934 can reuse thread local byte buffer
+        byte[] key = new byte[row.keySlice().capacity()];
+        row.keySlice().get(key);
+
+        return new SimpleDataRow(key, null);
+    }
+
+    /**
+     * Unpacks a raw value into (cur, old, ts) triplet. TODO asch IGNITE-15934 not very efficient.
+     *
+     * @param row The row.
+     * @return The value.
+     */
+    private static Value unpack(@Nullable DataRow row) {
+        if (row == null) {
+            return new Value(null, null, null);
+        }
+
+        ByteBuffer buf = row.value();
+
+        BinaryRow newVal = null;
+        BinaryRow oldVal = null;
+
+        int l1 = buf.asIntBuffer().get();
+
+        int pos = 4;
+
+        buf.position(pos);
+
+        if (l1 != 0) {
+            // TODO asch IGNITE-15934 get rid of copying
+            byte[] tmp = new byte[l1];
+
+            buf.get(tmp);
+
+            newVal = new ByteBufferRow(tmp);
+
+            pos += l1;
+        }
+
+        buf.position(pos);
+
+        int l2 = buf.asIntBuffer().get();
+
+        pos += 4;
+
+        buf.position(pos);
+
+        if (l2 != 0) {
+            // TODO asch get rid of copying
+            byte[] tmp = new byte[l2];
+
+            buf.get(tmp);
+
+            oldVal = new ByteBufferRow(tmp);
+
+            pos += l2;
+        }
+
+        buf.position(pos);
+
+        long ts = buf.getLong();
+        long nodeId = buf.getLong();
+
+        return new Value(newVal, oldVal, new Timestamp(ts, nodeId));
+    }
+
+    /**
+     * Packs a multi-versioned value.
+     *
+     * @param key The key.
+     * @param value The value.
+     * @return Data row.
+     */
+    private static DataRow pack(SearchRow key, Value value) {
+        byte[] b1 = null;
+        byte[] b2 = null;
+
+        int l1 = value.newRow == null ? 0 : (b1 = value.newRow.bytes()).length;
+        int l2 = value.oldRow == null ? 0 : (b2 = value.oldRow.bytes()).length;
+
+        // TODO asch write only values.
+        ByteBuffer buf = ByteBuffer.allocate(4 + l1 + 4 + l2 + 16);
+
+        buf.asIntBuffer().put(l1);
+
+        buf.position(4);
+
+        if (l1 > 0) {
+            buf.put(b1);
+        }
+
+        buf.asIntBuffer().put(l2);
+
+        buf.position(buf.position() + 4);
+
+        if (l2 > 0) {
+            buf.put(b2);
+        }
+
+        buf.putLong(value.timestamp.getTimestamp());
+        buf.putLong(value.timestamp.getNodeId());
+
+        return new SimpleDataRow(key.keyBytes(), buf.array());
+    }
+
+    /**
+     * Resolves a multi-versioned value depending on a viewer's timestamp.
+     *
+     * @param val        The value.
+     * @param timestamp  The timestamp
+     * @return New and old rows pair.
+     * @see #versionedRow
+     */
+    private Pair<BinaryRow, BinaryRow> resolve(Value val, Timestamp timestamp) {
+        if (val.timestamp == null) { // New or after reset.
+            assert val.oldRow == null : val;
+
+            return new Pair<>(val.newRow, null);
+        }
+
+        // Checks "inTx" condition. Will be false if this is a first transactional op.
+        if (val.timestamp.equals(timestamp)) {
+            return new Pair<>(val.newRow, val.oldRow);
+        }
+
+        TxState state = txManager.state(val.timestamp);
+
+        BinaryRow cur;
+
+        if (state == TxState.ABORTED) { // Was aborted and had written a temp value.
+            cur = val.oldRow;
+        } else {
+            cur = val.newRow;
+        }
+
+        return new Pair<>(cur, cur);
+    }
+
+    /**
+     * Takes a snapshot.
+     *
+     * @param path The path.
+     * @return Snapshot future.
+     */
+    public CompletionStage<Void> snapshot(Path path) {
+        return storage.snapshot(path);
+    }
+
+    /**
+     * Restores a snapshot.
+     *
+     * @param path The path.
+     */
+    public void restoreSnapshot(Path path) {
+        storage.restoreSnapshot(path);
+    }
+
+    /**
+     * Executes a scan.
+     *
+     * @param pred The predicate.
+     * @return The cursor.
+     */
+    public Cursor<BinaryRow> scan(Predicate<SearchRow> pred) {
+        Cursor<DataRow> delegate = storage.scan(pred);
+
+        // TODO asch add tx support IGNITE-15087.
+        return new Cursor<BinaryRow>() {
+            private @Nullable BinaryRow cur = null;
+
+            @Override
+            public void close() throws Exception {
+                delegate.close();
+            }
+
+            @NotNull
+            @Override
+            public Iterator<BinaryRow> iterator() {
+                return this;
+            }
+
+            @Override
+            public boolean hasNext() {
+                if (cur != null) {
+                    return true;
+                }
+
+                if (delegate.hasNext()) {
+                    DataRow row = delegate.next();
+
+                    cur = versionedRow(row, null).getFirst();
+
+                    return cur != null ? true : hasNext(); // Skip tombstones.
+                }
+
+                return false;
+            }
+
+            @Override
+            public BinaryRow next() {
+                BinaryRow next = cur;
+
+                cur = null;
+
+                assert next != null;
+
+                return next;
+            }
+        };
+    }
+
+    /**
+     * Versioned value.
+     */
+    private static class Value {
+        /** Current value. */
+        BinaryRow newRow;
+
+        /** The value for rollback. */
+        @Nullable BinaryRow oldRow;
+
+        /** Transaction's timestamp. */
+        Timestamp timestamp;
+
+        /**
+         * The constructor.
+         *
+         * @param newRow New row.
+         * @param oldRow Old row.
+         * @param timestamp The timestamp.
+         */
+        Value(@Nullable BinaryRow newRow, @Nullable BinaryRow oldRow, Timestamp timestamp) {
+            this.newRow = newRow;
+            this.oldRow = oldRow;
+            this.timestamp = timestamp;
+        }
+    }
+
+    /**
+     * Wrapper provides correct byte[] comparison.
+     */
+    public static class KeyWrapper {
+        /** Data. */
+        private final byte[] data;
+
+        /** Hash. */
+        private final int hash;
+
+        /**
+         * The constructor.
+         *
+         * @param data Wrapped data.
+         */
+        public KeyWrapper(byte[] data, int hash) {
+            assert data != null;
+
+            this.data = data;
+            this.hash = hash;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+
+            KeyWrapper wrapper = (KeyWrapper) o;
+            return Arrays.equals(data, wrapper.data);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public int hashCode() {
+            return hash;
+        }
+    }
+
+    /**
+     * Returns a storage delegate.
+     *
+     * @return The delegate.
+     */
+    public PartitionStorage delegate() {
+        return storage;
+    }
+
+    /**
+     * Returns a transaction manager.
+     *
+     * @return Transaction manager.
+     */
+    public TxManager txManager() {
+        return txManager;
+    }
+
+    /**
+     * Adapter that converts a {@link BinaryRow} into a {@link SearchRow}.
+     */
+    private static class BinarySearchRow implements SearchRow {
+        /** Search key. */
+        private final byte[] keyBytes;
+
+        /** Source row. */
+        private final BinaryRow sourceRow;
+
+        /**
+         * The constructor.
+         *
+         * @param row The search row.
+         */
+        BinarySearchRow(BinaryRow row) {
+            sourceRow = row;
... 7804 lines suppressed ...