You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2023/01/20 14:51:33 UTC

[ignite-3] branch main updated: IGNITE-18397 Rework Watches based on Raft Learners (#1490)

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

ibessonov 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 db8f1e38a5 IGNITE-18397 Rework Watches based on Raft Learners (#1490)
db8f1e38a5 is described below

commit db8f1e38a5e0fab7b2195dec91d8b9913a384b00
Author: Alexander Polovtcev <al...@gmail.com>
AuthorDate: Fri Jan 20 17:51:26 2023 +0300

    IGNITE-18397 Rework Watches based on Raft Learners (#1490)
---
 modules/distribution-zones/build.gradle            |   2 -
 .../distributionzones/DistributionZoneManager.java | 147 +++---
 ...ibutionZoneManagerConfigurationChangesTest.java |   4 +-
 ...butionZoneManagerLogicalTopologyEventsTest.java |   4 +-
 .../DistributionZoneManagerWatchListenerTest.java  |  15 +-
 .../internal/metastorage/MetaStorageManager.java   |  19 +-
 .../ignite/internal/metastorage/WatchEvent.java    |  30 +-
 .../ignite/internal/metastorage/WatchListener.java |  10 +-
 modules/metastorage/build.gradle                   |   1 +
 .../impl/ItMetaStorageManagerImplTest.java         |  11 +-
 .../impl/ItMetaStorageServicePersistenceTest.java  |  51 +-
 .../metastorage/impl/ItMetaStorageServiceTest.java | 532 +++++++++------------
 .../metastorage/impl/ItMetaStorageWatchTest.java   | 336 +++++++++++++
 .../server/raft/ItMetaStorageRaftGroupTest.java    |  77 ++-
 .../command/MetastorageCommandsMessageGroup.java   |   6 -
 .../metastorage/command/WatchExactKeysCommand.java |  82 ----
 .../metastorage/command/WatchRangeKeysCommand.java |  54 ---
 .../metastorage/impl/MetaStorageManagerImpl.java   | 515 ++++++++------------
 .../metastorage/impl/MetaStorageService.java       | 117 ++---
 .../metastorage/impl/MetaStorageServiceImpl.java   | 266 +----------
 .../metastorage/server/KeyValueStorage.java        |  54 ++-
 .../ignite/internal/metastorage/server/Watch.java  |  89 ++++
 .../server/persistence/RocksDbKeyValueStorage.java | 338 +++++++++++--
 .../server/persistence/WatchCursor.java            | 185 -------
 .../server/raft/MetaStorageLearnerListener.java    |  73 +++
 .../server/raft/MetaStorageListener.java           | 334 +------------
 .../server/raft/MetaStorageWriteHandler.java       | 245 ++++++++++
 .../metastorage/watch/AggregatedWatch.java         |  68 ---
 .../internal/metastorage/watch/KeyCriterion.java   | 369 --------------
 .../metastorage/watch/WatchAggregator.java         | 263 ----------
 .../impl/MetaStorageRangeCursorTest.java           |  12 +-
 .../server/AbstractKeyValueStorageTest.java        | 529 ++++++++++----------
 .../server/RocksDbKeyValueStorageTest.java         |  70 ++-
 .../server/SimpleInMemoryKeyValueStorageTest.java  |   2 +-
 .../metastorage/watch/WatchAggregatorTest.java     | 312 ------------
 .../server/SimpleInMemoryKeyValueStorage.java      | 437 ++++++++---------
 .../internal/raft/service/RaftGroupService.java    |   4 +-
 .../ignite/internal/raft/RaftGroupServiceImpl.java |   2 +-
 .../service/ItAbstractListenerSnapshotTest.java    |  27 +-
 .../ItDistributedConfigurationPropertiesTest.java  |  21 +-
 .../ItDistributedConfigurationStorageTest.java     |   8 +-
 .../storage/ItRebalanceDistributedTest.java        |   2 +-
 .../internal/runner/app/ItDataSchemaSyncTest.java  |   2 +-
 .../runner/app/ItIgniteNodeRestartTest.java        | 119 +++--
 .../internal/runner/app/ItTablesApiTest.java       |   2 -
 .../internal/sql/engine/ItIndexSpoolTest.java      |  30 +-
 .../internal/sql/engine/ItSecondaryIndexTest.java  |  34 +-
 .../ignite/internal/table/ItTableScanTest.java     |  36 +-
 .../internal/test/WatchListenerInhibitor.java      | 152 ++----
 .../org/apache/ignite/internal/app/IgniteImpl.java |   2 +-
 .../storage/DistributedConfigurationStorage.java   |  40 +-
 .../DistributedConfigurationCatchUpTest.java       |  11 +-
 .../DistributedConfigurationStorageTest.java       |   2 +-
 .../ignite/distributed/ItTablePersistenceTest.java |   7 +-
 .../internal/table/distributed/TableManager.java   |  24 +-
 55 files changed, 2604 insertions(+), 3580 deletions(-)

diff --git a/modules/distribution-zones/build.gradle b/modules/distribution-zones/build.gradle
index 5d7b34fb8b..5be2d24f47 100644
--- a/modules/distribution-zones/build.gradle
+++ b/modules/distribution-zones/build.gradle
@@ -32,8 +32,6 @@ dependencies {
     implementation project(':ignite-api')
     implementation project(':ignite-cluster-management')
     implementation project(':ignite-metastorage-api')
-    // TODO: remove this dependency, see https://issues.apache.org/jira/browse/IGNITE-18397
-    implementation project(':ignite-metastorage')
     implementation project(':ignite-vault')
     implementation project(':ignite-configuration')
     implementation project(':ignite-schema')
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
index 5d82f6cfae..cf9abba0c4 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
@@ -70,7 +70,6 @@ import org.apache.ignite.internal.metastorage.dsl.CompoundCondition;
 import org.apache.ignite.internal.metastorage.dsl.Condition;
 import org.apache.ignite.internal.metastorage.dsl.If;
 import org.apache.ignite.internal.metastorage.dsl.Update;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
 import org.apache.ignite.internal.schema.configuration.TableChange;
 import org.apache.ignite.internal.schema.configuration.TableConfiguration;
 import org.apache.ignite.internal.schema.configuration.TableView;
@@ -78,11 +77,11 @@ import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.network.ClusterNode;
-import org.jetbrains.annotations.NotNull;
 
 /**
  * Distribution zones manager.
@@ -141,8 +140,8 @@ public class DistributionZoneManager implements IgniteComponent {
      */
     private volatile Set<String> logicalTopology;
 
-    /** Watch listener id to unregister the watch listener on {@link DistributionZoneManager#stop()}. */
-    private volatile Long watchListenerId;
+    /** Watch listener. Needed to unregister it on {@link DistributionZoneManager#stop()}. */
+    private final WatchListener watchListener;
 
     /**
      * Creates a new distribution zone manager.
@@ -166,6 +165,8 @@ public class DistributionZoneManager implements IgniteComponent {
         this.logicalTopologyService = logicalTopologyService;
         this.vaultMgr = vaultMgr;
 
+        this.watchListener = createMetastorageListener();
+
         logicalTopology = Collections.emptySet();
     }
 
@@ -437,9 +438,11 @@ public class DistributionZoneManager implements IgniteComponent {
 
             logicalTopologyService.addEventListener(topologyEventListener);
 
-            registerMetaStorageWatchListener()
-                    .thenAccept(ignore -> initDataNodesFromVaultManager())
-                    .thenAccept(ignore -> initMetaStorageKeysOnStart());
+            metaStorageManager.registerExactWatch(zonesLogicalTopologyKey(), watchListener);
+
+            initDataNodesFromVaultManager();
+
+            initMetaStorageKeysOnStart();
         } finally {
             busyLock.leaveBusy();
         }
@@ -456,9 +459,7 @@ public class DistributionZoneManager implements IgniteComponent {
 
         logicalTopologyService.removeEventListener(topologyEventListener);
 
-        if (watchListenerId != null) {
-            metaStorageManager.unregisterWatch(watchListenerId);
-        }
+        metaStorageManager.unregisterWatch(watchListener);
     }
 
     private class ZonesConfigurationListener implements ConfigurationNamedListListener<DistributionZoneView> {
@@ -741,39 +742,27 @@ public class DistributionZoneManager implements IgniteComponent {
         }
 
         try {
-            // TODO: Remove this call as part of https://issues.apache.org/jira/browse/IGNITE-18397
-            vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
-                    .thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()))
-                    .thenAccept(vaultAppliedRevision -> {
+            long appliedRevision = metaStorageManager.appliedRevision();
+
+            vaultMgr.get(zonesLogicalTopologyKey())
+                    .thenAccept(vaultEntry -> {
                         if (!busyLock.enterBusy()) {
                             throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
                         }
 
                         try {
-                            vaultMgr.get(zonesLogicalTopologyKey())
-                                    .thenAccept(vaultEntry -> {
-                                        if (!busyLock.enterBusy()) {
-                                            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-                                        }
-
-                                        try {
-                                            if (vaultEntry != null && vaultEntry.value() != null) {
-                                                logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
-
-                                                zonesConfiguration.distributionZones().value().namedListKeys()
-                                                        .forEach(zoneName -> {
-                                                            int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId()
-                                                                    .value();
-
-                                                            saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), vaultAppliedRevision);
-                                                        });
-
-                                                saveDataNodesToMetaStorage(DEFAULT_ZONE_ID, vaultEntry.value(), vaultAppliedRevision);
-                                            }
-                                        } finally {
-                                            busyLock.leaveBusy();
-                                        }
-                                    });
+                            if (vaultEntry != null && vaultEntry.value() != null) {
+                                logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
+
+                                zonesConfiguration.distributionZones().value().namedListKeys()
+                                        .forEach(zoneName -> {
+                                            int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId().value();
+
+                                            saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), appliedRevision);
+                                        });
+
+                                saveDataNodesToMetaStorage(DEFAULT_ZONE_ID, vaultEntry.value(), appliedRevision);
+                            }
                         } finally {
                             busyLock.leaveBusy();
                         }
@@ -783,65 +772,59 @@ public class DistributionZoneManager implements IgniteComponent {
         }
     }
 
-    /**
-     * Registers {@link WatchListener} which updates data nodes of distribution zones on logical topology changing event.
-     *
-     * @return Future representing pending completion of the operation.
-     */
-    private CompletableFuture<?> registerMetaStorageWatchListener() {
-        return metaStorageManager.registerExactWatch(zonesLogicalTopologyKey(), new WatchListener() {
-                    @Override
-                    public boolean onUpdate(@NotNull WatchEvent evt) {
-                        if (!busyLock.enterBusy()) {
-                            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-                        }
+    private WatchListener createMetastorageListener() {
+        return new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent evt) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+                }
 
-                        try {
-                            assert evt.single() : "Expected an event with one entry but was an event with several entries with keys: "
-                                    + evt.entryEvents().stream().map(entry -> entry.newEntry() == null ? "null" : entry.newEntry().key())
-                                    .collect(toList());
+                try {
+                    assert evt.single() : "Expected an event with one entry but was an event with several entries with keys: "
+                            + evt.entryEvents().stream().map(entry -> entry.newEntry() == null ? "null" : entry.newEntry().key())
+                            .collect(toList());
 
-                            Entry newEntry = evt.entryEvent().newEntry();
+                    Entry newEntry = evt.entryEvent().newEntry();
 
-                            long revision = newEntry.revision();
+                    long revision = newEntry.revision();
 
-                            byte[] newLogicalTopologyBytes = newEntry.value();
+                    byte[] newLogicalTopologyBytes = newEntry.value();
 
-                            Set<String> newLogicalTopology = ByteUtils.fromBytes(newLogicalTopologyBytes);
+                    vaultMgr.put(new ByteArray(newEntry.key()), newLogicalTopologyBytes).join();
 
-                            List<String> removedNodes =
-                                    logicalTopology.stream().filter(node -> !newLogicalTopology.contains(node)).collect(toList());
+                    Set<String> newLogicalTopology = ByteUtils.fromBytes(newLogicalTopologyBytes);
 
-                            List<String> addedNodes =
-                                    newLogicalTopology.stream().filter(node -> !logicalTopology.contains(node)).collect(toList());
+                    List<String> removedNodes =
+                            logicalTopology.stream().filter(node -> !newLogicalTopology.contains(node)).collect(toList());
 
-                            logicalTopology = newLogicalTopology;
+                    List<String> addedNodes =
+                            newLogicalTopology.stream().filter(node -> !logicalTopology.contains(node)).collect(toList());
 
-                            NamedConfigurationTree<DistributionZoneConfiguration, DistributionZoneView, DistributionZoneChange> zones =
-                                    zonesConfiguration.distributionZones();
+                    logicalTopology = newLogicalTopology;
 
-                            for (int i = 0; i < zones.value().size(); i++) {
-                                DistributionZoneView zoneView = zones.value().get(i);
+                    NamedConfigurationTree<DistributionZoneConfiguration, DistributionZoneView, DistributionZoneChange> zones =
+                            zonesConfiguration.distributionZones();
 
-                                scheduleTimers(zoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
-                            }
+                    for (int i = 0; i < zones.value().size(); i++) {
+                        DistributionZoneView zoneView = zones.value().get(i);
 
-                            DistributionZoneView defaultZoneView = zonesConfiguration.value().defaultDistributionZone();
+                        scheduleTimers(zoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+                    }
 
-                            scheduleTimers(defaultZoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+                    DistributionZoneView defaultZoneView = zonesConfiguration.value().defaultDistributionZone();
 
-                            return true;
-                        } finally {
-                            busyLock.leaveBusy();
-                        }
-                    }
+                    scheduleTimers(defaultZoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
 
-                    @Override
-                    public void onError(@NotNull Throwable e) {
-                        LOG.warn("Unable to process logical topology event", e);
-                    }
-                })
-                .thenAccept(id -> watchListenerId = id);
+            @Override
+            public void onError(Throwable e) {
+                LOG.warn("Unable to process logical topology event", e);
+            }
+        };
     }
 
     private void scheduleTimers(
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
index f2787d6a64..ba1dea44be 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
@@ -115,8 +115,6 @@ public class DistributionZoneManagerConfigurationChangesTest extends IgniteAbstr
 
         MetaStorageManager metaStorageManager = mock(MetaStorageManager.class);
 
-        when(metaStorageManager.registerExactWatch(any(), any())).thenReturn(completedFuture(null));
-
         logicalTopologyService = mock(LogicalTopologyServiceImpl.class);
 
         vaultMgr = mock(VaultManager.class);
@@ -155,7 +153,7 @@ public class DistributionZoneManagerConfigurationChangesTest extends IgniteAbstr
 
         AtomicLong raftIndex = new AtomicLong();
 
-        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
 
         MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
 
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
index d9b50ec8a4..de897b0708 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
@@ -129,8 +129,6 @@ public class DistributionZoneManagerLogicalTopologyEventsTest {
 
         when(vaultMgr.get(any())).thenReturn(completedFuture(null));
 
-        when(metaStorageManager.registerExactWatch(any(), any())).then(invocation -> completedFuture(null));
-
         TablesConfiguration tablesConfiguration = mock(TablesConfiguration.class);
 
         NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = mock(NamedConfigurationTree.class);
@@ -155,7 +153,7 @@ public class DistributionZoneManagerLogicalTopologyEventsTest {
 
         AtomicLong raftIndex = new AtomicLong();
 
-        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
 
         MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
 
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
index 7e8480f9da..e36870e966 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
@@ -34,6 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.after;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.lenient;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -71,7 +72,6 @@ import org.apache.ignite.internal.metastorage.command.info.StatementResultInfo;
 import org.apache.ignite.internal.metastorage.dsl.If;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
 import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
 import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
 import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
 import org.apache.ignite.internal.raft.Command;
@@ -162,12 +162,13 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
         mockVaultAppliedRevision(1);
 
         when(vaultMgr.get(zonesLogicalTopologyKey())).thenReturn(completedFuture(new VaultEntry(zonesLogicalTopologyKey(), null)));
+        when(vaultMgr.put(any(), any())).thenReturn(completedFuture(null));
 
-        when(metaStorageManager.registerExactWatch(any(), any())).then(invocation -> {
+        doAnswer(invocation -> {
             watchListener = invocation.getArgument(1);
 
-            return completedFuture(null);
-        });
+            return null;
+        }).when(metaStorageManager).registerExactWatch(any(), any());
 
         mockDefaultZoneConfiguration();
         mockDefaultZoneView();
@@ -175,7 +176,7 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
 
         AtomicLong raftIndex = new AtomicLong();
 
-        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+        keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
 
         MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
 
@@ -496,8 +497,6 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
     }
 
     private void mockVaultAppliedRevision(long revision) {
-        // TODO: remove this as part of https://issues.apache.org/jira/browse/IGNITE-18397
-        when(vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV))
-                .thenReturn(completedFuture(new VaultEntry(MetaStorageManagerImpl.APPLIED_REV, longToBytes(revision))));
+        when(metaStorageManager.appliedRevision()).thenReturn(revision);
     }
 }
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index 14083fed82..d42203dcf2 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -37,6 +37,12 @@ import org.jetbrains.annotations.Nullable;
  * Manager that provides operations on the Meta Storage.
  */
 public interface MetaStorageManager extends IgniteComponent {
+    /**
+     * Returns the current <em>applied revision</em> of the Meta Storage, that is, the most recent revision of updates that have been
+     * applied on this node.
+     */
+    long appliedRevision();
+
     /**
      * Retrieves an entry for the given key.
      */
@@ -69,8 +75,6 @@ public interface MetaStorageManager extends IgniteComponent {
      * Retrieves entries for the given key prefix in lexicographic order. Entries will be filtered out by upper bound of given revision
      * number.
      *
-     * <p>Prefix query is a synonym of the range query {@code range(prefixKey, nextKey(prefixKey))}.
-     *
      * @param keyPrefix Prefix of the key to retrieve the entries. Couldn't be {@code null}.
      * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
      * @return Cursor built upon entries corresponding to the given range and revision.
@@ -106,18 +110,16 @@ public interface MetaStorageManager extends IgniteComponent {
      *
      * @param key Prefix to listen to.
      * @param lsnr Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
      */
-    CompletableFuture<Long> registerPrefixWatch(ByteArray key, WatchListener lsnr);
+    void registerPrefixWatch(ByteArray key, WatchListener lsnr);
 
     /**
      * Registers a watch listener for the provided key.
      *
      * @param key Meta Storage key.
      * @param listener Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
      */
-    CompletableFuture<Long> registerExactWatch(ByteArray key, WatchListener listener);
+    void registerExactWatch(ByteArray key, WatchListener listener);
 
     /**
      * Registers a watch listener by a key range.
@@ -125,14 +127,13 @@ public interface MetaStorageManager extends IgniteComponent {
      * @param keyFrom Start of the range (inclusive).
      * @param keyTo End of the range (exclusive) or {@code null} if the range doesn't have an upper bound.
      * @param listener Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
      */
-    CompletableFuture<Long> registerRangeWatch(ByteArray keyFrom, ByteArray keyTo, WatchListener listener);
+    void registerRangeWatch(ByteArray keyFrom, @Nullable ByteArray keyTo, WatchListener listener);
 
     /**
      * Unregisters a watch listener.
      */
-    CompletableFuture<Void> unregisterWatch(long id);
+    void unregisterWatch(WatchListener lsnr);
 
     /**
      * Starts all registered watches.
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
index 512882fe9f..6a2b19d592 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.metastorage;
 
 import java.util.Collection;
 import java.util.List;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.tostring.S;
 
 /**
  * Watch event contains all entry updates done under one revision. Each particular entry update in this revision is represented by {@link
@@ -29,19 +29,19 @@ public class WatchEvent {
     /** Events about each entry update in the revision. */
     private final List<EntryEvent> entryEvts;
 
-    /** Designates that watch event contains only one update revision. */
-    private final boolean single;
+    private final long revision;
 
     /**
      * Constructs an watch event with given entry events collection.
      *
      * @param entryEvts Events for entries corresponding to an update under one revision.
+     * @param revision Revision of the updated entries.
      */
-    public WatchEvent(List<EntryEvent> entryEvts) {
+    public WatchEvent(List<EntryEvent> entryEvts, long revision) {
         assert entryEvts != null && !entryEvts.isEmpty();
 
-        this.single = entryEvts.size() == 1;
         this.entryEvts = entryEvts;
+        this.revision = revision;
     }
 
     /**
@@ -49,8 +49,8 @@ public class WatchEvent {
      *
      * @param entryEvt Entry event.
      */
-    public WatchEvent(@NotNull EntryEvent entryEvt) {
-        this(List.of(entryEvt));
+    public WatchEvent(EntryEvent entryEvt) {
+        this(List.of(entryEvt), entryEvt.newEntry().revision());
     }
 
     /**
@@ -59,7 +59,7 @@ public class WatchEvent {
      * @return {@code True} if watch event contains only one entry event.
      */
     public boolean single() {
-        return single;
+        return entryEvts.size() == 1;
     }
 
     /**
@@ -79,4 +79,18 @@ public class WatchEvent {
     public EntryEvent entryEvent() {
         return entryEvts.get(0);
     }
+
+    /**
+     * Returns the revision of the modified entries.
+     *
+     * @return Event revision.
+     */
+    public long revision() {
+        return revision;
+    }
+
+    @Override
+    public String toString() {
+        return S.toString(this);
+    }
 }
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
index c8a03c75f7..a8a499b4c6 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.metastorage;
 
-import org.jetbrains.annotations.NotNull;
-
 /**
  * The listener which receives and handles watch updates.
  */
@@ -26,16 +24,14 @@ public interface WatchListener {
     /**
      * The method will be called on each meta storage update.
      *
-     * @param evt A single event or a batch. The batch always contains updates for specific revision.
-     * @return {@code True} if listener must continue event handling. If returns {@code false} then the listener and corresponding watch
-     *      will be unregistered.
+     * @param event A single event or a batch. The batch always contains updates for specific revision.
      */
-    boolean onUpdate(@NotNull WatchEvent evt);
+    void onUpdate(WatchEvent event);
 
     /**
      * The method will be called in case of an error occurred. The listener and corresponding watch will be unregistered.
      *
      * @param e Exception.
      */
-    void onError(@NotNull Throwable e);
+    void onError(Throwable e);
 }
diff --git a/modules/metastorage/build.gradle b/modules/metastorage/build.gradle
index 3895855176..28ebdd81a1 100644
--- a/modules/metastorage/build.gradle
+++ b/modules/metastorage/build.gradle
@@ -39,6 +39,7 @@ dependencies {
 
     testImplementation testFixtures(project(':ignite-core'))
     testImplementation libs.mockito.junit
+    testImplementation libs.hamcrest.core
 
     integrationTestImplementation libs.jetbrains.annotations
     integrationTestImplementation project(":ignite-cluster-management")
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
index 70ca2264fe..cc0c1accdb 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
@@ -23,6 +23,7 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutur
 import static org.apache.ignite.utils.ClusterServiceTestUtils.clusterService;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.contains;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -81,17 +82,23 @@ public class ItMetaStorageManagerImplTest {
 
         raftManager = new Loza(clusterService, raftConfiguration, workDir.resolve("loza"), new HybridClockImpl());
 
+        VaultManager vaultManager = mock(VaultManager.class);
+
+        when(vaultManager.get(any())).thenReturn(completedFuture(null));
+
         ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
 
         when(cmgManager.metaStorageNodes())
                 .thenReturn(completedFuture(Set.of(clusterService.localConfiguration().getName())));
 
+        var storage = new RocksDbKeyValueStorage(clusterService.localConfiguration().getName(), workDir.resolve("metastorage"));
+
         metaStorageManager = new MetaStorageManagerImpl(
-                mock(VaultManager.class),
+                vaultManager,
                 clusterService,
                 cmgManager,
                 raftManager,
-                new RocksDbKeyValueStorage(workDir.resolve("metastorage"))
+                storage
         );
 
         clusterService.start();
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
index c8f0ad554f..eaf3faecd0 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
@@ -21,17 +21,22 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.function.BooleanSupplier;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
 import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
 import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
+import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
 import org.apache.ignite.internal.raft.service.ItAbstractListenerSnapshotTest;
 import org.apache.ignite.internal.raft.service.RaftGroupListener;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
 import org.junit.jupiter.api.AfterEach;
 
@@ -49,20 +54,20 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
     private MetaStorageServiceImpl metaStorage;
 
-    private KeyValueStorage storage;
+    private final Map<String, RocksDbKeyValueStorage> storageByName = new HashMap<>();
 
     /** After each. */
     @AfterEach
-    void tearDown() {
-        if (storage != null) {
-            storage.close();
-        }
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(storageByName.values().stream().map(storage -> storage::close));
     }
 
     /** {@inheritDoc} */
     @Override
-    public void beforeFollowerStop(RaftGroupService service) throws Exception {
-        metaStorage = new MetaStorageServiceImpl(service, null, null);
+    public void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
+        ClusterNode followerNode = getNode(server);
+
+        metaStorage = new MetaStorageServiceImpl(service, followerNode);
 
         // Put some data in the metastorage
         metaStorage.put(FIRST_KEY, FIRST_VALUE).get();
@@ -73,7 +78,15 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
     /** {@inheritDoc} */
     @Override
-    public void afterFollowerStop(RaftGroupService service) throws Exception {
+    public void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
+        ClusterNode followerNode = getNode(server);
+
+        KeyValueStorage storage = storageByName.remove(followerNode.name());
+
+        if (storage != null) {
+            storage.close();
+        }
+
         // Remove the first key from the metastorage
         metaStorage.remove(FIRST_KEY).get();
 
@@ -96,7 +109,9 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
     /** {@inheritDoc} */
     @Override
     public BooleanSupplier snapshotCheckClosure(JraftServerImpl restarted, boolean interactedAfterSnapshot) {
-        KeyValueStorage storage = getListener(restarted, raftGroupId()).getStorage();
+        ClusterNode node = getNode(restarted);
+
+        KeyValueStorage storage = storageByName.get(node.name());
 
         byte[] lastKey = interactedAfterSnapshot ? SECOND_KEY.bytes() : FIRST_KEY.bytes();
         byte[] lastValue = interactedAfterSnapshot ? SECOND_VALUE : FIRST_VALUE;
@@ -111,16 +126,22 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
     /** {@inheritDoc} */
     @Override
-    public Path getListenerPersistencePath(MetaStorageListener listener) {
-        return ((RocksDbKeyValueStorage) listener.getStorage()).getDbPath();
+    public Path getListenerPersistencePath(MetaStorageListener listener, RaftServer server) {
+        return storageByName.get(getNode(server).name()).getDbPath();
     }
 
     /** {@inheritDoc} */
     @Override
     public RaftGroupListener createListener(ClusterService service, Path listenerPersistencePath) {
-        storage = new RocksDbKeyValueStorage(listenerPersistencePath);
+        String nodeName = service.localConfiguration().getName();
+
+        KeyValueStorage storage = storageByName.computeIfAbsent(nodeName, name -> {
+            var s = new RocksDbKeyValueStorage(name, listenerPersistencePath);
 
-        storage.start();
+            s.start();
+
+            return s;
+        });
 
         return new MetaStorageListener(storage);
     }
@@ -145,4 +166,8 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
 
         assertEquals(expected, entry);
     }
+
+    private static ClusterNode getNode(RaftServer server) {
+        return server.clusterService().topologyService().localMember();
+    }
 }
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
index ef21661168..78c8fda92a 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.metastorage.impl;
 
-import static java.util.concurrent.CompletableFuture.allOf;
 import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toUnmodifiableSet;
 import static org.apache.ignite.internal.metastorage.dsl.CompoundCondition.and;
 import static org.apache.ignite.internal.metastorage.dsl.CompoundCondition.or;
 import static org.apache.ignite.internal.metastorage.dsl.Conditions.revision;
@@ -31,13 +31,11 @@ import static org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTe
 import static org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTest.ServerUpdateMatcher.upd;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.apache.ignite.utils.ClusterServiceTestUtils.findLocalAddresses;
-import static org.apache.ignite.utils.ClusterServiceTestUtils.waitForTopology;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.eq;
@@ -60,20 +58,13 @@ import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.hlc.HybridClockImpl;
-import org.apache.ignite.internal.logger.IgniteLogger;
-import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Condition;
 import org.apache.ignite.internal.metastorage.dsl.Conditions;
 import org.apache.ignite.internal.metastorage.dsl.If;
@@ -92,6 +83,7 @@ import org.apache.ignite.internal.metastorage.server.OrCondition;
 import org.apache.ignite.internal.metastorage.server.RevisionCondition;
 import org.apache.ignite.internal.metastorage.server.ValueCondition;
 import org.apache.ignite.internal.metastorage.server.ValueCondition.Type;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageLearnerListener;
 import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
 import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
 import org.apache.ignite.internal.raft.Loza;
@@ -101,21 +93,21 @@ import org.apache.ignite.internal.raft.RaftGroupEventsListener;
 import org.apache.ignite.internal.raft.RaftManager;
 import org.apache.ignite.internal.raft.RaftNodeId;
 import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
+import org.apache.ignite.internal.raft.service.RaftGroupListener;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
 import org.apache.ignite.utils.ClusterServiceTestUtils;
 import org.hamcrest.Description;
 import org.hamcrest.TypeSafeMatcher;
-import org.jetbrains.annotations.NotNull;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
@@ -123,35 +115,23 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.ArgumentCaptor;
-import org.mockito.Mock;
-import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
  * Meta storage client tests.
  */
 @ExtendWith(WorkDirectoryExtension.class)
 @ExtendWith(ConfigurationExtension.class)
-@ExtendWith(MockitoExtension.class)
 public class ItMetaStorageServiceTest {
-    /** The logger. */
-    private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageServiceTest.class);
-
     /** Base network port. */
     private static final int NODE_PORT_BASE = 20_000;
 
-    /** Nodes. */
-    private static final int NODES = 2;
-
-    /**
-     * Expected server result entry.
-     */
-    private static final EntryImpl EXPECTED_RESULT_ENTRY =
-            new EntryImpl(
-                    new byte[]{1},
-                    new byte[]{2},
-                    10,
-                    2
-            );
+    /** Expected result entry. */
+    private static final Entry EXPECTED_RESULT_ENTRY = new EntryImpl(
+            new byte[]{1},
+            new byte[]{2},
+            10,
+            2
+    );
 
     /**
      * Expected result map.
@@ -161,38 +141,10 @@ public class ItMetaStorageServiceTest {
     /** Expected server result collection. */
     private static final Collection<Entry> EXPECTED_SRV_RESULT_COLL;
 
-    /** Node 0 id. */
-    private static final String NODE_ID_0 = "node-id-0";
-
-    /** Node 1 id. */
-    private static final String NODE_ID_1 = "node-id-1";
-
-    /** Cluster. */
-    private final ArrayList<ClusterService> cluster = new ArrayList<>();
-
-    /** Meta storage raft server. */
-    private final List<RaftManager> raftManagers = new ArrayList<>();
-
-    /** List of Raft services (for resource management purposes). */
-    private final List<RaftGroupService> raftGroupServices = new ArrayList<>();
-
-    /** Mock Metastorage storage. */
-    @Mock
-    private KeyValueStorage mockStorage;
-
-    /** Metastorage service. */
-    private MetaStorageService metaStorageSvc;
-
-    @WorkDirectory
-    private Path dataPath;
-
-    @InjectConfiguration
-    private RaftConfiguration raftConfiguration;
-
     static {
         EXPECTED_RESULT_MAP = new TreeMap<>();
 
-        EntryImpl entry1 = new EntryImpl(
+        Entry entry1 = new EntryImpl(
                 new byte[]{1},
                 new byte[]{2},
                 10,
@@ -201,7 +153,7 @@ public class ItMetaStorageServiceTest {
 
         EXPECTED_RESULT_MAP.put(new ByteArray(entry1.key()), entry1);
 
-        EntryImpl entry2 = new EntryImpl(
+        Entry entry2 = new EntryImpl(
                 new byte[]{3},
                 new byte[]{4},
                 10,
@@ -213,29 +165,115 @@ public class ItMetaStorageServiceTest {
         EXPECTED_SRV_RESULT_COLL = List.of(entry1, entry2);
     }
 
-    /**
-     * Run {@code NODES} cluster nodes.
-     */
+    private static class Node {
+        private final ClusterService clusterService;
+
+        private final RaftManager raftManager;
+
+        private final KeyValueStorage mockStorage;
+
+        private RaftGroupService metaStorageRaftService;
+
+        private MetaStorageService metaStorageService;
+
+        Node(ClusterService clusterService, RaftConfiguration raftConfiguration, Path dataPath) {
+            this.clusterService = clusterService;
+
+            this.raftManager = new Loza(
+                    clusterService,
+                    raftConfiguration,
+                    dataPath.resolve(name()),
+                    new HybridClockImpl()
+            );
+
+            this.mockStorage = mock(KeyValueStorage.class);
+        }
+
+        void start(PeersAndLearners configuration) {
+            clusterService.start();
+            raftManager.start();
+
+            CompletableFuture<RaftGroupService> raftService = startRaftService(configuration);
+
+            assertThat(raftService, willCompleteSuccessfully());
+
+            metaStorageRaftService = raftService.join();
+
+            ClusterNode node = clusterService.topologyService().localMember();
+
+            metaStorageService = new MetaStorageServiceImpl(metaStorageRaftService, node);
+        }
+
+        String name() {
+            return clusterService.localConfiguration().getName();
+        }
+
+        private CompletableFuture<RaftGroupService> startRaftService(PeersAndLearners configuration) {
+            String name = name();
+
+            boolean isLearner = configuration.peer(name) == null;
+
+            Peer peer = isLearner ? configuration.learner(name) : configuration.peer(name);
+
+            assert peer != null;
+
+            RaftGroupListener listener = isLearner ? new MetaStorageLearnerListener(mockStorage) : new MetaStorageListener(mockStorage);
+
+            var raftNodeId = new RaftNodeId(MetastorageGroupId.INSTANCE, peer);
+
+            try {
+                return raftManager.startRaftGroupNode(raftNodeId, configuration, listener, RaftGroupEventsListener.noopLsnr);
+            } catch (NodeStoppingException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        void stop() throws Exception {
+            Stream<AutoCloseable> raftStop = Stream.of(
+                    metaStorageRaftService == null ? null : (AutoCloseable) metaStorageRaftService::shutdown,
+                    () -> raftManager.stopRaftNodes(MetastorageGroupId.INSTANCE)
+            );
+
+            Stream<AutoCloseable> beforeNodeStop = Stream.of(raftManager, clusterService).map(c -> c::beforeNodeStop);
+
+            Stream<AutoCloseable> nodeStop = Stream.of(raftManager, clusterService).map(c -> c::stop);
+
+            IgniteUtils.closeAll(Stream.of(raftStop, beforeNodeStop, nodeStop).flatMap(Function.identity()));
+        }
+    }
+
+    private TestInfo testInfo;
+
+    @WorkDirectory
+    private Path workDir;
+
+    @InjectConfiguration
+    private RaftConfiguration raftConfiguration;
+
+    private final List<Node> nodes = new ArrayList<>();
+
     @BeforeEach
-    public void beforeTest(TestInfo testInfo) throws Exception {
-        List<NetworkAddress> localAddresses = findLocalAddresses(NODE_PORT_BASE, NODE_PORT_BASE + NODES);
+    public void beforeTest(TestInfo testInfo) {
+        this.testInfo = testInfo;
+    }
+
+    private List<Node> startNodes(int amount) {
+        List<NetworkAddress> localAddresses = findLocalAddresses(NODE_PORT_BASE, NODE_PORT_BASE + amount);
 
         var nodeFinder = new StaticNodeFinder(localAddresses);
 
         localAddresses.stream()
                 .map(addr -> ClusterServiceTestUtils.clusterService(testInfo, addr.port(), nodeFinder))
-                .forEach(clusterService -> {
-                    clusterService.start();
-                    cluster.add(clusterService);
-                });
+                .forEach(clusterService -> nodes.add(new Node(clusterService, raftConfiguration, workDir)));
 
-        for (ClusterService node : cluster) {
-            assertTrue(waitForTopology(node, NODES, 1000));
-        }
+        PeersAndLearners metaStorageConfiguration = PeersAndLearners.fromConsistentIds(
+                Set.of(nodes.get(0).name()),
+                nodes.stream().skip(1).map(Node::name).collect(toUnmodifiableSet())
+        );
 
-        LOG.info("Cluster started.");
+        nodes.parallelStream().forEach(node -> node.start(metaStorageConfiguration));
 
-        metaStorageSvc = prepareMetaStorage();
+        return nodes;
     }
 
     /**
@@ -245,16 +283,7 @@ public class ItMetaStorageServiceTest {
      */
     @AfterEach
     public void afterTest() throws Exception {
-        Stream<AutoCloseable> stopRaftGroupServices = raftGroupServices.stream().map(service -> service::shutdown);
-
-        Stream<AutoCloseable> stopRaftGroups = raftManagers.stream()
-                .map(manager -> () -> manager.stopRaftNodes(MetastorageGroupId.INSTANCE));
-
-        Stream<AutoCloseable> beforeNodeStop = Stream.concat(raftManagers.stream(), cluster.stream()).map(c -> c::beforeNodeStop);
-
-        Stream<AutoCloseable> nodeStop = Stream.concat(raftManagers.stream(), cluster.stream()).map(c -> c::stop);
-
-        IgniteUtils.closeAll(Stream.of(stopRaftGroupServices, stopRaftGroups, beforeNodeStop, nodeStop).flatMap(Function.identity()));
+        IgniteUtils.closeAll(nodes.stream().map(node -> node::stop));
     }
 
     /**
@@ -264,9 +293,11 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGet() throws Exception {
-        when(mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenReturn(EXPECTED_RESULT_ENTRY);
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenReturn(EXPECTED_RESULT_ENTRY);
 
-        assertEquals(EXPECTED_RESULT_ENTRY, metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+        assertEquals(EXPECTED_RESULT_ENTRY, node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
     }
 
     /**
@@ -276,11 +307,13 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetWithUpperBoundRevision() throws Exception {
-        when(mockStorage.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision())).thenReturn(EXPECTED_RESULT_ENTRY);
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision())).thenReturn(EXPECTED_RESULT_ENTRY);
 
         assertEquals(
                 EXPECTED_RESULT_ENTRY,
-                metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key()), EXPECTED_RESULT_ENTRY.revision()).get()
+                node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key()), EXPECTED_RESULT_ENTRY.revision()).get()
         );
     }
 
@@ -291,9 +324,11 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAll() throws Exception {
-        when(mockStorage.getAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.getAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
 
-        assertEquals(EXPECTED_RESULT_MAP, metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet()).get());
+        assertEquals(EXPECTED_RESULT_MAP, node.metaStorageService.getAll(EXPECTED_RESULT_MAP.keySet()).get());
     }
 
     /**
@@ -303,11 +338,13 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAllWithUpperBoundRevision() throws Exception {
-        when(mockStorage.getAll(anyList(), eq(10L))).thenReturn(EXPECTED_SRV_RESULT_COLL);
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.getAll(anyList(), eq(10L))).thenReturn(EXPECTED_SRV_RESULT_COLL);
 
         assertEquals(
                 EXPECTED_RESULT_MAP,
-                metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
+                node.metaStorageService.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
         );
     }
 
@@ -318,13 +355,15 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testPut() throws Exception {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKey = new ByteArray(new byte[]{1});
 
         byte[] expVal = {2};
 
-        doNothing().when(mockStorage).put(expKey.bytes(), expVal);
+        doNothing().when(node.mockStorage).put(expKey.bytes(), expVal);
 
-        metaStorageSvc.put(expKey, expVal).get();
+        node.metaStorageService.put(expKey, expVal).get();
     }
 
     /**
@@ -334,13 +373,15 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAndPut() throws Exception {
+        Node node = startNodes(1).get(0);
+
         byte[] expVal = {2};
 
-        when(mockStorage.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal)).thenReturn(EXPECTED_RESULT_ENTRY);
+        when(node.mockStorage.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal)).thenReturn(EXPECTED_RESULT_ENTRY);
 
         assertEquals(
                 EXPECTED_RESULT_ENTRY,
-                metaStorageSvc.getAndPut(new ByteArray(EXPECTED_RESULT_ENTRY.key()), expVal).get()
+                node.metaStorageService.getAndPut(new ByteArray(EXPECTED_RESULT_ENTRY.key()), expVal).get()
         );
     }
 
@@ -351,7 +392,9 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testPutAll() throws Exception {
-        metaStorageSvc.putAll(
+        Node node = startNodes(1).get(0);
+
+        node.metaStorageService.putAll(
                 EXPECTED_RESULT_MAP.entrySet().stream()
                         .collect(Collectors.toMap(
                                 Map.Entry::getKey,
@@ -362,7 +405,7 @@ public class ItMetaStorageServiceTest {
         ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
         ArgumentCaptor<List<byte[]>> valuesCaptor = ArgumentCaptor.forClass(List.class);
 
-        verify(mockStorage).putAll(keysCaptor.capture(), valuesCaptor.capture());
+        verify(node.mockStorage).putAll(keysCaptor.capture(), valuesCaptor.capture());
 
         // Assert keys equality.
         assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -392,9 +435,11 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAndPutAll() throws Exception {
-        when(mockStorage.getAndPutAll(anyList(), anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.getAndPutAll(anyList(), anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
 
-        Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndPutAll(
+        Map<ByteArray, Entry> gotRes = node.metaStorageService.getAndPutAll(
                 EXPECTED_RESULT_MAP.entrySet().stream()
                         .collect(Collectors.toMap(
                                 Map.Entry::getKey,
@@ -407,7 +452,7 @@ public class ItMetaStorageServiceTest {
         ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
         ArgumentCaptor<List<byte[]>> valuesCaptor = ArgumentCaptor.forClass(List.class);
 
-        verify(mockStorage).getAndPutAll(keysCaptor.capture(), valuesCaptor.capture());
+        verify(node.mockStorage).getAndPutAll(keysCaptor.capture(), valuesCaptor.capture());
 
         // Assert keys equality.
         assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -437,11 +482,13 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRemove() throws Exception {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKey = new ByteArray(new byte[]{1});
 
-        doNothing().when(mockStorage).remove(expKey.bytes());
+        doNothing().when(node.mockStorage).remove(expKey.bytes());
 
-        metaStorageSvc.remove(expKey).get();
+        node.metaStorageService.remove(expKey).get();
     }
 
     /**
@@ -451,16 +498,18 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAndRemove() throws Exception {
-        EntryImpl expRes = new EntryImpl(
+        Node node = startNodes(1).get(0);
+
+        Entry expRes = new EntryImpl(
                 new byte[]{1},
                 new byte[]{3},
                 10,
                 2
         );
 
-        when(mockStorage.getAndRemove(expRes.key())).thenReturn(expRes);
+        when(node.mockStorage.getAndRemove(expRes.key())).thenReturn(expRes);
 
-        assertEquals(expRes, metaStorageSvc.getAndRemove(new ByteArray(expRes.key())).get());
+        assertEquals(expRes, node.metaStorageService.getAndRemove(new ByteArray(expRes.key())).get());
     }
 
     /**
@@ -470,16 +519,16 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRemoveAll() throws Exception {
-        doNothing().when(mockStorage).removeAll(anyList());
+        Node node = startNodes(1).get(0);
 
-        metaStorageSvc.removeAll(EXPECTED_RESULT_MAP.keySet()).get();
+        node.metaStorageService.removeAll(EXPECTED_RESULT_MAP.keySet()).get();
 
         List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream()
                 .map(ByteArray::bytes).collect(toList());
 
         ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
 
-        verify(mockStorage).removeAll(keysCaptor.capture());
+        verify(node.mockStorage).removeAll(keysCaptor.capture());
 
         assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
 
@@ -495,15 +544,17 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testGetAndRemoveAll() throws Exception {
-        when(mockStorage.getAndRemoveAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+        Node node = startNodes(1).get(0);
 
-        Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
+        when(node.mockStorage.getAndRemoveAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+
+        Map<ByteArray, Entry> gotRes = node.metaStorageService.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
 
         assertEquals(EXPECTED_RESULT_MAP, gotRes);
 
         ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
 
-        verify(mockStorage).getAndRemoveAll(keysCaptor.capture());
+        verify(node.mockStorage).getAndRemoveAll(keysCaptor.capture());
 
         // Assert keys equality.
         assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -522,15 +573,17 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeWitKeyToAndUpperBound() {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
         ByteArray expKeyTo = new ByteArray(new byte[]{3});
 
         long expRevUpperBound = 10;
 
-        when(mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), expRevUpperBound, false)).thenReturn(mock(Cursor.class));
+        when(node.mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), expRevUpperBound, false)).thenReturn(mock(Cursor.class));
 
-        metaStorageSvc.range(expKeyFrom, expKeyTo, expRevUpperBound).close();
+        node.metaStorageService.range(expKeyFrom, expKeyTo, expRevUpperBound).close();
     }
 
     /**
@@ -539,13 +592,15 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeWitKeyTo() {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
         ByteArray expKeyTo = new ByteArray(new byte[]{3});
 
-        when(mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), false)).thenReturn(mock(Cursor.class));
+        when(node.mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), false)).thenReturn(mock(Cursor.class));
 
-        metaStorageSvc.range(expKeyFrom, expKeyTo).close();
+        node.metaStorageService.range(expKeyFrom, expKeyTo).close();
     }
 
     /**
@@ -554,11 +609,13 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeWitNullAsKeyTo() {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
-        when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(mock(Cursor.class));
+        when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(mock(Cursor.class));
 
-        metaStorageSvc.range(expKeyFrom, null).close();
+        node.metaStorageService.range(expKeyFrom, null).close();
     }
 
     /**
@@ -566,9 +623,11 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeHasNext() {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
-        when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenAnswer(invocation -> {
+        when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenAnswer(invocation -> {
             var cursor = mock(Cursor.class);
 
             when(cursor.hasNext()).thenReturn(true);
@@ -576,7 +635,7 @@ public class ItMetaStorageServiceTest {
             return cursor;
         });
 
-        Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
+        Cursor<Entry> cursor = node.metaStorageService.range(expKeyFrom, null);
 
         assertTrue(cursor.iterator().hasNext());
     }
@@ -586,7 +645,9 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeNext() {
-        when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
             var cursor = mock(Cursor.class);
 
             when(cursor.hasNext()).thenReturn(true);
@@ -595,7 +656,7 @@ public class ItMetaStorageServiceTest {
             return cursor;
         });
 
-        Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+        Cursor<Entry> cursor = node.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
 
         assertEquals(EXPECTED_RESULT_ENTRY, cursor.iterator().next());
     }
@@ -605,7 +666,9 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeNextNoSuchElementException() {
-        when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
             var cursor = mock(Cursor.class);
 
             when(cursor.hasNext()).thenReturn(true);
@@ -614,7 +677,7 @@ public class ItMetaStorageServiceTest {
             return cursor;
         });
 
-        Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+        Cursor<Entry> cursor = node.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
 
         assertThrows(NoSuchElementException.class, () -> cursor.iterator().next());
     }
@@ -625,124 +688,25 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testRangeClose() {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
-        Cursor cursorMock = mock(Cursor.class);
+        Cursor<Entry> cursorMock = mock(Cursor.class);
 
-        when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(cursorMock);
+        when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(cursorMock);
 
-        Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
+        Cursor<Entry> cursor = node.metaStorageService.range(expKeyFrom, null);
 
         cursor.close();
 
         verify(cursorMock, times(1)).close();
     }
 
-    @Test
-    public void testWatchOnUpdate() throws Exception {
-        WatchEvent expectedEvent =
-                new WatchEvent(List.of(
-                        new EntryEvent(
-                                new EntryImpl(
-                                        new byte[]{2},
-                                        new byte[]{20},
-                                        1,
-                                        1
-                                ),
-                                new EntryImpl(
-                                        new byte[]{2},
-                                        new byte[]{21},
-                                        2,
-                                        4
-                                )
-                        ),
-                        new EntryEvent(
-                                new EntryImpl(
-                                        new byte[]{3},
-                                        new byte[]{20},
-                                        1,
-                                        2
-                                ),
-                                new EntryImpl(
-                                        new byte[]{3},
-                                        new byte[]{},
-                                        2,
-                                        5
-                                )
-                        ),
-                        new EntryEvent(
-                                new EntryImpl(
-                                        new byte[]{4},
-                                        new byte[]{20},
-                                        1,
-                                        3
-                                ),
-                                new EntryImpl(
-                                        new byte[]{4},
-                                        new byte[]{},
-                                        3,
-                                        6
-                                )
-                        )
-                ));
-
-        ByteArray keyFrom = new ByteArray(new byte[]{1});
-
-        ByteArray keyTo = new ByteArray(new byte[]{10});
-
-        long rev = 2;
-
-        when(mockStorage.watch(keyFrom.bytes(), keyTo.bytes(), rev)).thenAnswer(invocation -> {
-            var cursor = mock(Cursor.class);
-
-            when(cursor.hasNext()).thenReturn(true);
-            when(cursor.next()).thenReturn(expectedEvent);
-
-            return cursor;
-        });
-
-        CountDownLatch latch = new CountDownLatch(1);
-
-        IgniteUuid watchId = metaStorageSvc.watch(keyFrom, keyTo, rev, new WatchListener() {
-            @Override
-            public boolean onUpdate(@NotNull WatchEvent event) {
-                Collection<EntryEvent> expectedEvents = expectedEvent.entryEvents();
-                Collection<EntryEvent> actualEvents = event.entryEvents();
-
-                assertEquals(expectedEvents.size(), actualEvents.size());
-
-                Iterator<EntryEvent> expectedIterator = expectedEvents.iterator();
-                Iterator<EntryEvent> actualIterator = actualEvents.iterator();
-
-                while (expectedIterator.hasNext() && actualIterator.hasNext()) {
-                    EntryEvent expectedEntryEvent = expectedIterator.next();
-                    EntryEvent actualEntryEvent = actualIterator.next();
-
-                    assertArrayEquals(expectedEntryEvent.oldEntry().key(), actualEntryEvent.oldEntry().key());
-                    assertArrayEquals(expectedEntryEvent.oldEntry().value(), actualEntryEvent.oldEntry().value());
-                    assertArrayEquals(expectedEntryEvent.newEntry().key(), actualEntryEvent.newEntry().key());
-                    assertArrayEquals(expectedEntryEvent.newEntry().value(), actualEntryEvent.newEntry().value());
-                }
-
-                latch.countDown();
-
-                return true;
-            }
-
-            @Override
-            public void onError(@NotNull Throwable e) {
-                // Within given test it's not expected to get here.
-                fail();
-            }
-        }).get();
-
-        latch.await();
-
-        metaStorageSvc.stopWatch(watchId).get();
-    }
-
     @Test
     public void testMultiInvoke() throws Exception {
+        Node node = startNodes(1).get(0);
+
         ByteArray key1 = new ByteArray(new byte[]{1});
         ByteArray key2 = new ByteArray(new byte[]{2});
         ByteArray key3 = new ByteArray(new byte[]{3});
@@ -780,11 +744,11 @@ public class ItMetaStorageServiceTest {
 
         var ifCaptor = ArgumentCaptor.forClass(org.apache.ignite.internal.metastorage.server.If.class);
 
-        when(mockStorage.invoke(any())).thenReturn(new StatementResult(true));
+        when(node.mockStorage.invoke(any())).thenReturn(new StatementResult(true));
 
-        assertTrue(metaStorageSvc.invoke(iif).get().getAsBoolean());
+        assertTrue(node.metaStorageService.invoke(iif).get().getAsBoolean());
 
-        verify(mockStorage).invoke(ifCaptor.capture());
+        verify(node.mockStorage).invoke(ifCaptor.capture());
 
         var resultIf = ifCaptor.getValue();
 
@@ -818,11 +782,13 @@ public class ItMetaStorageServiceTest {
 
     @Test
     public void testInvoke() throws Exception {
+        Node node = startNodes(1).get(0);
+
         ByteArray expKey = new ByteArray(new byte[]{1});
 
         byte[] expVal = {2};
 
-        when(mockStorage.invoke(any(), any(), any())).thenReturn(true);
+        when(node.mockStorage.invoke(any(), any(), any())).thenReturn(true);
 
         Condition condition = Conditions.notExists(expKey);
 
@@ -830,7 +796,7 @@ public class ItMetaStorageServiceTest {
 
         Operation failure = Operations.noop();
 
-        assertTrue(metaStorageSvc.invoke(condition, success, failure).get());
+        assertTrue(node.metaStorageService.invoke(condition, success, failure).get());
 
         var conditionCaptor = ArgumentCaptor.forClass(AbstractSimpleCondition.class);
 
@@ -838,7 +804,7 @@ public class ItMetaStorageServiceTest {
 
         ArgumentCaptor<Collection<Operation>> failureCaptor = ArgumentCaptor.forClass(Collection.class);
 
-        verify(mockStorage).invoke(conditionCaptor.capture(), successCaptor.capture(), failureCaptor.capture());
+        verify(node.mockStorage).invoke(conditionCaptor.capture(), successCaptor.capture(), failureCaptor.capture());
 
         assertArrayEquals(expKey.bytes(), conditionCaptor.getValue().key());
 
@@ -857,10 +823,11 @@ public class ItMetaStorageServiceTest {
     @Disabled // TODO: IGNITE-14693 Add tests for exception handling logic.
     @Test
     public void testGetThatThrowsCompactedException() {
-        when(mockStorage.get(EXPECTED_RESULT_ENTRY.key()))
-                .thenThrow(new org.apache.ignite.internal.metastorage.server.CompactedException());
+        Node node = startNodes(1).get(0);
 
-        assertThrows(CompactedException.class, () -> metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+        when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new CompactedException());
+
+        assertThrows(CompactedException.class, () -> node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
     }
 
     /**
@@ -869,9 +836,11 @@ public class ItMetaStorageServiceTest {
     @Disabled // TODO: IGNITE-14693 Add tests for exception handling logic.
     @Test
     public void testGetThatThrowsOperationTimeoutException() {
-        when(mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new OperationTimeoutException());
+        Node node = startNodes(1).get(0);
+
+        when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new OperationTimeoutException());
 
-        assertThrows(OperationTimeoutException.class, () -> metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+        assertThrows(OperationTimeoutException.class, () -> node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
     }
 
     /**
@@ -881,7 +850,12 @@ public class ItMetaStorageServiceTest {
      */
     @Test
     public void testCursorsCleanup() throws Exception {
-        when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+        startNodes(2);
+
+        Node leader = nodes.get(0);
+        Node learner = nodes.get(1);
+
+        when(leader.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
             var cursor = mock(Cursor.class);
 
             when(cursor.hasNext()).thenReturn(true);
@@ -890,31 +864,19 @@ public class ItMetaStorageServiceTest {
             return cursor;
         });
 
-        String localName = cluster.get(0).topologyService().localMember().name();
-
-        PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(Set.of(localName));
-
-        RaftGroupService metaStorageRaftSvc2 = raftManagers.get(1)
-                .startRaftGroupService(MetastorageGroupId.INSTANCE, configuration)
-                .get(3, TimeUnit.SECONDS);
-
-        raftGroupServices.add(metaStorageRaftSvc2);
-
-        MetaStorageService metaStorageSvc2 = new MetaStorageServiceImpl(metaStorageRaftSvc2, NODE_ID_1, NODE_ID_1);
-
-        Cursor<Entry> cursorNode0 = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+        Cursor<Entry> cursorNode0 = leader.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
 
         assertTrue(cursorNode0.hasNext());
 
-        Cursor<Entry> cursor2Node0 = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+        Cursor<Entry> cursor2Node0 = leader.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
 
         assertTrue(cursor2Node0.hasNext());
 
-        Cursor<Entry> cursorNode1 = metaStorageSvc2.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+        Cursor<Entry> cursorNode1 = learner.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
 
         assertTrue(cursorNode1.hasNext());
 
-        metaStorageSvc.closeCursors(NODE_ID_0).get();
+        leader.metaStorageService.closeCursors(leader.clusterService.topologyService().localMember().id()).get();
 
         assertThrows(NoSuchElementException.class, () -> cursorNode0.iterator().next());
 
@@ -976,7 +938,7 @@ public class ItMetaStorageServiceTest {
     }
 
     /**
-     * Matcher for {@link org.apache.ignite.internal.metastorage.server.Condition}.
+     * Matcher for {@link Condition}.
      */
     protected static class ServerConditionMatcher extends TypeSafeMatcher<org.apache.ignite.internal.metastorage.server.Condition> {
 
@@ -1029,54 +991,4 @@ public class ItMetaStorageServiceTest {
 
         }
     }
-
-    /**
-     * Prepares meta storage by instantiating corresponding raft server with {@link MetaStorageListener} and {@link
-     * MetaStorageServiceImpl}.
-     *
-     * @return {@link MetaStorageService} instance.
-     */
-    private MetaStorageService prepareMetaStorage() throws Exception {
-        String localName = cluster.get(0).topologyService().localMember().name();
-
-        PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(Set.of(localName));
-
-        CompletableFuture<RaftGroupService> firstService = startRaftService(cluster.get(0), configuration);
-        CompletableFuture<RaftGroupService> secondService = startRaftService(cluster.get(1), configuration);
-
-        assertThat(allOf(firstService, secondService), willCompleteSuccessfully());
-
-        raftGroupServices.add(firstService.join());
-        raftGroupServices.add(secondService.join());
-
-        RaftGroupService metaStorageRaftGrpSvc = secondService.join();
-
-        return new MetaStorageServiceImpl(metaStorageRaftGrpSvc, NODE_ID_0, NODE_ID_0);
-    }
-
-    private CompletableFuture<RaftGroupService> startRaftService(
-            ClusterService node, PeersAndLearners configuration
-    ) throws NodeStoppingException {
-        var raftManager = new Loza(
-                node,
-                raftConfiguration,
-                dataPath.resolve("raftManager" + raftManagers.size()),
-                new HybridClockImpl());
-
-        raftManager.start();
-
-        raftManagers.add(raftManager);
-
-        Peer serverPeer = configuration.peer(node.topologyService().localMember().name());
-
-        if (serverPeer == null) {
-            return raftManager.startRaftGroupService(MetastorageGroupId.INSTANCE, configuration);
-        } else {
-            var nodeId = new RaftNodeId(MetastorageGroupId.INSTANCE, serverPeer);
-
-            return raftManager.startRaftGroupNode(
-                    nodeId, configuration, new MetaStorageListener(mockStorage), RaftGroupEventsListener.noopLsnr
-            );
-        }
-    }
 }
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java
new file mode 100644
index 0000000000..2c8772adda
--- /dev/null
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.metastorage.impl;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.utils.ClusterServiceTestUtils.findLocalAddresses;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.metastorage.dsl.Conditions;
+import org.apache.ignite.internal.metastorage.dsl.Operations;
+import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.StaticNodeFinder;
+import org.apache.ignite.utils.ClusterServiceTestUtils;
+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;
+
+/**
+ * Tests for Meta Storage Watches.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
+public class ItMetaStorageWatchTest {
+    private static class Node {
+        private final ClusterService clusterService;
+
+        private final RaftManager raftManager;
+
+        private final MetaStorageManager metaStorageManager;
+
+        private final CompletableFuture<Set<String>> metaStorageNodesFuture = new CompletableFuture<>();
+
+        Node(ClusterService clusterService, RaftConfiguration raftConfiguration, Path dataPath) {
+            this.clusterService = clusterService;
+
+            Path basePath = dataPath.resolve(name());
+
+            this.raftManager = new Loza(
+                    clusterService,
+                    raftConfiguration,
+                    basePath.resolve("raft"),
+                    new HybridClockImpl()
+            );
+
+            var vaultManager = mock(VaultManager.class);
+
+            when(vaultManager.get(any())).thenReturn(CompletableFuture.completedFuture(null));
+            when(vaultManager.put(any(), any())).thenReturn(CompletableFuture.completedFuture(null));
+
+            var cmgManager = mock(ClusterManagementGroupManager.class);
+
+            when(cmgManager.metaStorageNodes()).thenReturn(metaStorageNodesFuture);
+
+            this.metaStorageManager = new MetaStorageManagerImpl(
+                    vaultManager,
+                    clusterService,
+                    cmgManager,
+                    raftManager,
+                    new RocksDbKeyValueStorage(name(), basePath.resolve("storage"))
+            );
+        }
+
+        void start(Set<String> metaStorageNodes) {
+            clusterService.start();
+            raftManager.start();
+            metaStorageManager.start();
+
+            metaStorageNodesFuture.complete(metaStorageNodes);
+        }
+
+        String name() {
+            return clusterService.localConfiguration().getName();
+        }
+
+        void stop() throws Exception {
+            Stream<AutoCloseable> beforeNodeStop = Stream.of(metaStorageManager, raftManager, clusterService).map(c -> c::beforeNodeStop);
+
+            Stream<AutoCloseable> nodeStop = Stream.of(metaStorageManager, raftManager, clusterService).map(c -> c::stop);
+
+            IgniteUtils.closeAll(Stream.concat(beforeNodeStop, nodeStop));
+        }
+    }
+
+    private TestInfo testInfo;
+
+    @WorkDirectory
+    private Path workDir;
+
+    @InjectConfiguration
+    private RaftConfiguration raftConfiguration;
+
+    private final List<Node> nodes = new ArrayList<>();
+
+    @BeforeEach
+    public void beforeTest(TestInfo testInfo) {
+        this.testInfo = testInfo;
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(nodes.stream().map(node -> node::stop));
+    }
+
+    private void startNodes(int amount) {
+        List<NetworkAddress> localAddresses = findLocalAddresses(10_000, 10_000 + nodes.size() + amount);
+
+        var nodeFinder = new StaticNodeFinder(localAddresses);
+
+        localAddresses.stream()
+                .map(addr -> ClusterServiceTestUtils.clusterService(testInfo, addr.port(), nodeFinder))
+                .forEach(clusterService -> nodes.add(new Node(clusterService, raftConfiguration, workDir)));
+
+        nodes.parallelStream().forEach(node -> node.start(Set.of(nodes.get(0).name())));
+    }
+
+    @Test
+    void testExactWatch() throws Exception {
+        testWatches((node, latch) -> node.metaStorageManager.registerExactWatch(new ByteArray("foo"), new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent event) {
+                assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+                assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+                latch.countDown();
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                fail();
+            }
+        }));
+    }
+
+    @Test
+    void testPrefixWatch() throws Exception {
+        testWatches((node, latch) -> node.metaStorageManager.registerPrefixWatch(new ByteArray("fo"), new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent event) {
+                assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+                assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+                latch.countDown();
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                fail();
+            }
+        }));
+    }
+
+    @Test
+    void testRangeWatch() throws Exception {
+        testWatches((node, latch) -> {
+            var startRange = new ByteArray("fo" + ('o' - 1));
+            var endRange = new ByteArray("foz");
+
+            node.metaStorageManager.registerRangeWatch(startRange, endRange, new WatchListener() {
+                @Override
+                public void onUpdate(WatchEvent event) {
+                    assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+                    assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+                    latch.countDown();
+                }
+
+                @Override
+                public void onError(Throwable e) {
+                    fail();
+                }
+            });
+        });
+    }
+
+    private void testWatches(BiConsumer<Node, CountDownLatch> registerWatchAction) throws Exception {
+        int numNodes = 3;
+
+        startNodes(numNodes);
+
+        var latch = new CountDownLatch(numNodes);
+
+        for (Node node : nodes) {
+            registerWatchAction.accept(node, latch);
+
+            node.metaStorageManager.deployWatches();
+        }
+
+        var key = new ByteArray("foo");
+
+        CompletableFuture<Boolean> invokeFuture = nodes.get(0).metaStorageManager.invoke(
+                Conditions.notExists(key),
+                Operations.put(key, "bar".getBytes(StandardCharsets.UTF_8)),
+                Operations.noop()
+        );
+
+        assertThat(invokeFuture, willBe(true));
+
+        assertTrue(latch.await(10, TimeUnit.SECONDS));
+    }
+
+    /**
+     * Tests that metastorage missed metastorage events are replayed after deploying watches.
+     */
+    @Test
+    void testReplayUpdates() throws InterruptedException {
+        int numNodes = 3;
+
+        startNodes(numNodes);
+
+        var exactLatch = new CountDownLatch(numNodes);
+        var prefixLatch = new CountDownLatch(numNodes);
+
+        for (Node node : nodes) {
+            node.metaStorageManager.registerExactWatch(new ByteArray("foo"), new WatchListener() {
+                @Override
+                public void onUpdate(WatchEvent event) {
+                    assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+                    assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+                    exactLatch.countDown();
+                }
+
+                @Override
+                public void onError(Throwable e) {
+                    fail();
+                }
+            });
+
+            node.metaStorageManager.registerPrefixWatch(new ByteArray("ba"), new WatchListener() {
+                @Override
+                public void onUpdate(WatchEvent event) {
+                    List<String> keys = event.entryEvents().stream()
+                            .map(e -> new String(e.newEntry().key(), StandardCharsets.UTF_8))
+                            .collect(Collectors.toList());
+
+                    List<String> values = event.entryEvents().stream()
+                            .map(e -> new String(e.newEntry().value(), StandardCharsets.UTF_8))
+                            .collect(Collectors.toList());
+
+                    assertThat(keys, containsInAnyOrder("bar", "baz"));
+                    assertThat(values, containsInAnyOrder("one", "two"));
+
+                    prefixLatch.countDown();
+                }
+
+                @Override
+                public void onError(Throwable e) {
+                    fail();
+                }
+            });
+        }
+
+        CompletableFuture<Boolean> invokeFuture = nodes.get(0).metaStorageManager.invoke(
+                Conditions.notExists(new ByteArray("foo")),
+                Operations.put(new ByteArray("foo"), "bar".getBytes(StandardCharsets.UTF_8)),
+                Operations.noop()
+        );
+
+        assertThat(invokeFuture, willBe(true));
+
+        invokeFuture = nodes.get(0).metaStorageManager.invoke(
+                Conditions.exists(new ByteArray("foo")),
+                List.of(
+                        Operations.put(new ByteArray("bar"), "one".getBytes(StandardCharsets.UTF_8)),
+                        Operations.put(new ByteArray("baz"), "two".getBytes(StandardCharsets.UTF_8))
+                ),
+                List.of()
+        );
+
+        assertThat(invokeFuture, willBe(true));
+
+        nodes.forEach(node -> {
+            try {
+                node.metaStorageManager.deployWatches();
+            } catch (NodeStoppingException e) {
+                throw new RuntimeException(e);
+            }
+        });
+
+        assertTrue(exactLatch.await(10, TimeUnit.SECONDS));
+        assertTrue(prefixLatch.await(10, TimeUnit.SECONDS));
+    }
+}
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
index af270fa3e7..3c26c3d8b3 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
@@ -26,14 +26,12 @@ import static org.apache.ignite.utils.ClusterServiceTestUtils.waitForTopology;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.when;
 
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -44,7 +42,6 @@ import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTest;
 import org.apache.ignite.internal.metastorage.impl.MetaStorageService;
 import org.apache.ignite.internal.metastorage.impl.MetaStorageServiceImpl;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
@@ -84,7 +81,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
 @ExtendWith(MockitoExtension.class)
 public class ItMetaStorageRaftGroupTest {
     /** The logger. */
-    private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageServiceTest.class);
+    private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageRaftGroupTest.class);
 
     /** Base network port. */
     private static final int NODE_PORT_BASE = 20_000;
@@ -215,8 +212,7 @@ public class ItMetaStorageRaftGroupTest {
         final AtomicInteger replicatorStoppedCounter = new AtomicInteger(0);
 
         when(mockStorage.range(EXPECTED_RESULT_ENTRY1.key(), new byte[]{4}, false)).thenAnswer(invocation -> {
-            List<Entry> entries = new ArrayList<>(
-                    List.of(EXPECTED_RESULT_ENTRY1, EXPECTED_RESULT_ENTRY2));
+            List<Entry> entries = List.of(EXPECTED_RESULT_ENTRY1, EXPECTED_RESULT_ENTRY2);
 
             return Cursor.fromBareIterator(entries.iterator());
         });
@@ -228,55 +224,58 @@ public class ItMetaStorageRaftGroupTest {
 
         String oldLeaderId = raftServersRaftGroups.get(0).value.leader().consistentId();
 
-        Optional<RaftServer> oldLeaderServer = raftServers.stream()
-                .filter(s -> localMemberName(s.clusterService()).equals(oldLeaderId)).findFirst();
+        RaftServer oldLeaderServer = raftServers.stream()
+                .filter(s -> localMemberName(s.clusterService()).equals(oldLeaderId))
+                .findFirst()
+                .orElseThrow();
 
         //Server that will be alive after we stop leader.
-        Optional<RaftServer> liveServer = raftServers.stream()
-                .filter(s -> !localMemberName(s.clusterService()).equals(oldLeaderId)).findFirst();
-
-        if (oldLeaderServer.isEmpty() || liveServer.isEmpty()) {
-            fail();
-        }
+        RaftServer liveServer = raftServers.stream()
+                .filter(s -> !localMemberName(s.clusterService()).equals(oldLeaderId))
+                .findFirst()
+                .orElseThrow();
 
         RaftGroupService raftGroupServiceOfLiveServer = raftServersRaftGroups.stream()
-                .filter(p -> p.key.equals(liveServer.get()))
-                .findFirst().get().value;
+                .filter(p -> p.key.equals(liveServer))
+                .findFirst()
+                .orElseThrow()
+                .value;
 
         MetaStorageService metaStorageSvc = new MetaStorageServiceImpl(
-                raftGroupServiceOfLiveServer, "some_node", "some_node");
+                raftGroupServiceOfLiveServer, liveServer.clusterService().topologyService().localMember());
 
-        Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY1.key()), new ByteArray(new byte[]{4}));
+        try (Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY1.key()), new ByteArray(new byte[]{4}))) {
 
-        assertTrue(waitForCondition(
-                () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
+            assertTrue(waitForCondition(
+                    () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
 
-        assertTrue(cursor.hasNext());
+            assertTrue(cursor.hasNext());
 
-        assertEquals(EXPECTED_RESULT_ENTRY1, (cursor.iterator().next()));
+            assertEquals(EXPECTED_RESULT_ENTRY1, (cursor.iterator().next()));
 
-        // Ensure that leader has not been changed.
-        // In a stable topology unexpected leader election shouldn't happen.
-        assertTrue(waitForCondition(
-                () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
+            // Ensure that leader has not been changed.
+            // In a stable topology unexpected leader election shouldn't happen.
+            assertTrue(waitForCondition(
+                    () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
 
-        //stop leader
-        oldLeaderServer.get().stopRaftNodes(MetastorageGroupId.INSTANCE);
-        oldLeaderServer.get().stop();
-        cluster.stream().filter(c -> localMemberName(c).equals(oldLeaderId)).findFirst().get().stop();
+            //stop leader
+            oldLeaderServer.stopRaftNodes(MetastorageGroupId.INSTANCE);
+            oldLeaderServer.stop();
+            cluster.stream().filter(c -> localMemberName(c).equals(oldLeaderId)).findFirst().orElseThrow().stop();
 
-        raftGroupServiceOfLiveServer.refreshLeader().get();
+            raftGroupServiceOfLiveServer.refreshLeader().get();
 
-        assertNotSame(oldLeaderId, raftGroupServiceOfLiveServer.leader().consistentId());
+            assertNotSame(oldLeaderId, raftGroupServiceOfLiveServer.leader().consistentId());
 
-        // ensure that leader has been changed only once
-        assertTrue(waitForCondition(
-                () -> replicatorStartedCounter.get() == 4, 5_000), replicatorStartedCounter.get() + "");
-        assertTrue(waitForCondition(
-                () -> replicatorStoppedCounter.get() == 2, 5_000), replicatorStoppedCounter.get() + "");
+            // ensure that leader has been changed only once
+            assertTrue(waitForCondition(
+                    () -> replicatorStartedCounter.get() == 4, 5_000), replicatorStartedCounter.get() + "");
+            assertTrue(waitForCondition(
+                    () -> replicatorStoppedCounter.get() == 2, 5_000), replicatorStoppedCounter.get() + "");
 
-        assertTrue(cursor.hasNext());
-        assertEquals(EXPECTED_RESULT_ENTRY2, (cursor.iterator().next()));
+            assertTrue(cursor.hasNext());
+            assertEquals(EXPECTED_RESULT_ENTRY2, (cursor.iterator().next()));
+        }
     }
 
     private List<Pair<RaftServer, RaftGroupService>> prepareJraftMetaStorages(AtomicInteger replicatorStartedCounter,
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
index 0166e56618..40f44346c1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
@@ -100,12 +100,6 @@ public interface MetastorageCommandsMessageGroup {
     /** Message type for {@link PrefixCommand}. */
     short PREFIX = 61;
 
-    /** Message type for {@link WatchExactKeysCommand}. */
-    short WATCH_EXACT_KEYS = 70;
-
-    /** Message type for {@link WatchRangeKeysCommand}. */
-    short WATCH_RANGE_KEYS = 71;
-
     /** Message type for {@link CursorHasNextCommand}. */
     short CURSOR_HAS_NEXT = 80;
 
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java
deleted file mode 100644
index 1d94c68f19..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java
+++ /dev/null
@@ -1,82 +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.internal.metastorage.command;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import org.apache.ignite.internal.raft.WriteCommand;
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
- */
-@Transferable(MetastorageCommandsMessageGroup.WATCH_EXACT_KEYS)
-public interface WatchExactKeysCommand extends WriteCommand {
-    /**
-     * Returns the keys list. Couldn't be {@code null}.
-     */
-    List<byte[]> keys();
-
-    /**
-     * Returns start revision inclusive. {@code 0} - all revisions.
-     */
-    long revision();
-
-    /**
-     * Returns id of the node that requests range.
-     */
-    String requesterNodeId();
-
-    /**
-     * Returns id of cursor that is associated with the current command.
-     */
-    IgniteUuid cursorId();
-
-    /**
-     * Static constructor.
-     *
-     * @param commandsFactory Commands factory.
-     * @param keys            The keys collection. Couldn't be {@code null}.
-     * @param revision        Start revision inclusive. {@code 0} - all revisions.
-     * @param requesterNodeId Id of the node that requests watch.
-     * @param cursorId        Id of cursor that is associated with the current command.
-     */
-    static WatchExactKeysCommand watchExactKeysCommand(
-            MetaStorageCommandsFactory commandsFactory,
-            Set<ByteArray> keys,
-            long revision,
-            String requesterNodeId,
-            IgniteUuid cursorId
-    ) {
-        List<byte[]> list = new ArrayList<>(keys.size());
-
-        for (ByteArray key : keys) {
-            list.add(key.bytes());
-        }
-
-        return commandsFactory.watchExactKeysCommand()
-                .keys(list)
-                .revision(revision)
-                .requesterNodeId(requesterNodeId)
-                .cursorId(cursorId)
-                .build();
-    }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java
deleted file mode 100644
index 32cc85ace8..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java
+++ /dev/null
@@ -1,54 +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.internal.metastorage.command;
-
-import org.apache.ignite.internal.raft.WriteCommand;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.network.annotations.Transferable;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
- */
-@Transferable(MetastorageCommandsMessageGroup.WATCH_RANGE_KEYS)
-public interface WatchRangeKeysCommand extends WriteCommand {
-    /**
-     * Returns start key of range (inclusive). Couldn't be {@code null}.
-     */
-    byte @Nullable [] keyFrom();
-
-    /**
-     * Returns end key of range (exclusive). Could be {@code null}.
-     */
-    byte @Nullable [] keyTo();
-
-    /**
-     * Returns start revision inclusive. {@code 0} - all revisions.
-     */
-    long revision();
-
-    /**
-     * Returns id of the node that requests range.
-     */
-    String requesterNodeId();
-
-    /**
-     * Returns id of cursor that is associated with the current command.
-     */
-    IgniteUuid cursorId();
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
index 4b41865fa8..a6ae8c8278 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
@@ -17,14 +17,16 @@
 
 package org.apache.ignite.internal.metastorage.impl;
 
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.stream.Collectors.toSet;
 import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
 import static org.apache.ignite.internal.util.ByteUtils.longToBytes;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_EXECUTION_ERR;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.DEPLOYING_WATCH_ERR;
 
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
@@ -32,6 +34,8 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.metastorage.WatchListener;
@@ -43,25 +47,21 @@ import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
 import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
 import org.apache.ignite.internal.metastorage.exceptions.OperationTimeoutException;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageLearnerListener;
 import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
 import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
-import org.apache.ignite.internal.metastorage.watch.AggregatedWatch;
-import org.apache.ignite.internal.metastorage.watch.KeyCriterion;
-import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
 import org.apache.ignite.internal.raft.Peer;
 import org.apache.ignite.internal.raft.PeersAndLearners;
 import org.apache.ignite.internal.raft.RaftGroupEventsListener;
 import org.apache.ignite.internal.raft.RaftManager;
 import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.Status;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
@@ -80,11 +80,12 @@ import org.jetbrains.annotations.Nullable;
  * </ul>
  */
 public class MetaStorageManagerImpl implements MetaStorageManager {
+    private static final IgniteLogger LOG = Loggers.forClass(MetaStorageManagerImpl.class);
+
     /**
-     * Special key for the vault where the applied revision for {@link MetaStorageManagerImpl#storeEntries} operation is stored.
-     * This mechanism is needed for committing processed watches to {@link VaultManager}.
+     * Special key for the vault where the applied revision for {@link MetaStorageManagerImpl#saveRevision} operation is stored.
      */
-    public static final ByteArray APPLIED_REV = ByteArray.fromString("applied_revision");
+    private static final ByteArray APPLIED_REV = ByteArray.fromString("applied_revision");
 
     private final ClusterService clusterService;
 
@@ -97,22 +98,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
     private final ClusterManagementGroupManager cmgMgr;
 
     /** Meta storage service. */
-    private volatile CompletableFuture<MetaStorageService> metaStorageSvcFut;
-
-    /**
-     * Aggregator of multiple watches to deploy them as one batch.
-     *
-     * @see WatchAggregator
-     */
-    private final WatchAggregator watchAggregator = new WatchAggregator();
-
-    /**
-     * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
-     * {@code null} if no watch deployed at the moment.
-     *
-     * <p>Multi-threaded access is guarded by {@code this}.
-     */
-    private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
+    private volatile CompletableFuture<MetaStorageServiceImpl> metaStorageSvcFut;
 
     /** Actual storage for the Metastorage. */
     private final KeyValueStorage storage;
@@ -120,22 +106,13 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
-    /**
-     * If true - all new watches will be deployed immediately.
-     *
-     * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
-     *
-     * <p>Multi-threaded access is guarded by {@code this}.
-     */
-    private boolean areWatchesDeployed = false;
+    /** Prevents double stopping of the component. */
+    private final AtomicBoolean isStopped = new AtomicBoolean();
 
     /**
-     * Flag that indicates that the component has been initialized.
+     * Applied revision of the Meta Storage, that is, the most recent revision that has been processed on this node.
      */
-    private volatile boolean isInitialized = false;
-
-    /** Prevents double stopping of the component. */
-    private final AtomicBoolean isStopped = new AtomicBoolean();
+    private volatile long appliedRevision;
 
     /**
      * The constructor.
@@ -159,32 +136,58 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         this.storage = storage;
     }
 
-    private CompletableFuture<MetaStorageService> initializeMetaStorage(Set<String> metaStorageNodes) {
+    private CompletableFuture<MetaStorageServiceImpl> initializeMetaStorage(Set<String> metaStorageNodes) {
         ClusterNode thisNode = clusterService.topologyService().localMember();
 
-        PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes);
-
-        Peer localPeer = configuration.peer(thisNode.name());
+        String thisNodeName = thisNode.name();
 
         CompletableFuture<RaftGroupService> raftServiceFuture;
 
         try {
-            if (localPeer == null) {
-                raftServiceFuture = raftMgr.startRaftGroupService(MetastorageGroupId.INSTANCE, configuration);
-            } else {
-                clusterService.topologyService().addEventHandler(new TopologyEventHandler() {
-                    @Override
-                    public void onDisappeared(ClusterNode member) {
-                        metaStorageSvcFut.thenAccept(svc -> svc.closeCursors(member.id()));
-                    }
-                });
+            // We need to configure the replication protocol differently whether this node is a synchronous or asynchronous replica.
+            if (metaStorageNodes.contains(thisNodeName)) {
+                PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes);
+
+                Peer localPeer = configuration.peer(thisNodeName);
 
-                storage.start();
+                assert localPeer != null;
 
                 raftServiceFuture = raftMgr.startRaftGroupNode(
                         new RaftNodeId(MetastorageGroupId.INSTANCE, localPeer),
                         configuration,
                         new MetaStorageListener(storage),
+                        new RaftGroupEventsListener() {
+                            @Override
+                            public void onLeaderElected(long term) {
+                                // TODO: add listener to remove learners when they leave Logical Topology
+                                //  see https://issues.apache.org/jira/browse/IGNITE-18554
+
+                                registerTopologyEventListener();
+
+                                // Update the configuration immediately in case we missed some updates.
+                                addLearners(clusterService.topologyService().allMembers());
+                            }
+
+                            @Override
+                            public void onNewPeersConfigurationApplied(PeersAndLearners configuration) {
+                            }
+
+                            @Override
+                            public void onReconfigurationError(Status status, PeersAndLearners configuration, long term) {
+                            }
+                        }
+                );
+            } else {
+                PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes, Set.of(thisNodeName));
+
+                Peer localPeer = configuration.learner(thisNodeName);
+
+                assert localPeer != null;
+
+                raftServiceFuture = raftMgr.startRaftGroupNode(
+                        new RaftNodeId(MetastorageGroupId.INSTANCE, localPeer),
+                        configuration,
+                        new MetaStorageLearnerListener(storage),
                         RaftGroupEventsListener.noopLsnr
                 );
             }
@@ -192,12 +195,97 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
             return CompletableFuture.failedFuture(e);
         }
 
-        return raftServiceFuture.thenApply(service -> new MetaStorageServiceImpl(service, thisNode.id(), thisNode.name()));
+        return raftServiceFuture.thenApply(raftService -> new MetaStorageServiceImpl(raftService, thisNode));
+    }
+
+    private void registerTopologyEventListener() {
+        clusterService.topologyService().addEventHandler(new TopologyEventHandler() {
+            @Override
+            public void onAppeared(ClusterNode member) {
+                addLearners(List.of(member));
+            }
+
+            @Override
+            public void onDisappeared(ClusterNode member) {
+                metaStorageSvcFut.thenAccept(service -> isCurrentNodeLeader(service.raftGroupService())
+                        .thenAccept(isLeader -> {
+                            if (isLeader) {
+                                service.closeCursors(member.id());
+                            }
+                        }));
+            }
+        });
+    }
+
+    private void addLearners(Collection<ClusterNode> nodes) {
+        if (!busyLock.enterBusy()) {
+            LOG.info("Skipping Meta Storage configuration update because the node is stopping");
+
+            return;
+        }
+
+        try {
+            metaStorageSvcFut
+                    .thenApply(MetaStorageServiceImpl::raftGroupService)
+                    .thenCompose(raftService -> isCurrentNodeLeader(raftService)
+                            .thenCompose(isLeader -> isLeader ? addLearners(raftService, nodes) : completedFuture(null)))
+                    .whenComplete((v, e) -> {
+                        if (e != null) {
+                            LOG.error("Unable to change peers on topology update", e);
+                        }
+                    });
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private CompletableFuture<Void> addLearners(RaftGroupService raftService, Collection<ClusterNode> nodes) {
+        if (!busyLock.enterBusy()) {
+            LOG.info("Skipping Meta Storage configuration update because the node is stopping");
+
+            return completedFuture(null);
+        }
+
+        try {
+            Set<String> peers = raftService.peers().stream()
+                    .map(Peer::consistentId)
+                    .collect(toSet());
+
+            Set<String> learners = nodes.stream()
+                    .map(ClusterNode::name)
+                    .filter(name -> !peers.contains(name))
+                    .collect(toSet());
+
+            if (learners.isEmpty()) {
+                return completedFuture(null);
+            }
+
+            if (LOG.isInfoEnabled()) {
+                LOG.info("New Meta Storage learners detected: " + learners);
+            }
+
+            PeersAndLearners newConfiguration = PeersAndLearners.fromConsistentIds(peers, learners);
+
+            return raftService.addLearners(newConfiguration.learners());
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    private CompletableFuture<Boolean> isCurrentNodeLeader(RaftGroupService raftService) {
+        String name = clusterService.topologyService().localMember().name();
+
+        return raftService.refreshLeader()
+                .thenApply(v -> raftService.leader().consistentId().equals(name));
     }
 
     /** {@inheritDoc} */
     @Override
     public void start() {
+        this.appliedRevision = readAppliedRevision().join();
+
+        storage.start();
+
         this.metaStorageSvcFut = cmgMgr.metaStorageNodes()
                 .thenCompose(metaStorageNodes -> {
                     if (!busyLock.enterBusy()) {
@@ -205,8 +293,6 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
                     }
 
                     try {
-                        isInitialized = true;
-
                         return initializeMetaStorage(metaStorageNodes);
                     } finally {
                         busyLock.leaveBusy();
@@ -223,135 +309,59 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
 
         busyLock.block();
 
-        if (!isInitialized) {
-            // Stop command was called before the init command was received
-            return;
-        }
-
-        synchronized (this) {
-            IgniteUtils.closeAll(
-                    this::stopDeployedWatches,
-                    () -> {
-                        if (raftMgr.stopRaftNodes(MetastorageGroupId.INSTANCE)) {
-                            storage.close();
-                        }
-                    }
-            );
-        }
-    }
+        metaStorageSvcFut.cancel(true);
 
-    private void stopDeployedWatches() throws Exception {
-        if (!areWatchesDeployed) {
-            return;
-        }
-
-        deployFut
-                .thenCompose(watchId -> watchId == null
-                        ? CompletableFuture.completedFuture(null)
-                        : metaStorageSvcFut.thenAccept(service -> service.stopWatch(watchId))
-                )
-                .get();
+        IgniteUtils.closeAll(
+                () -> raftMgr.stopRaftNodes(MetastorageGroupId.INSTANCE),
+                storage::close
+        );
     }
 
     @Override
-    public synchronized void deployWatches() throws NodeStoppingException {
-        if (!busyLock.enterBusy()) {
-            throw new NodeStoppingException();
-        }
-
-        try {
-            CompletableFuture<IgniteUuid> deployFut = this.deployFut;
-
-            updateAggregatedWatch().whenComplete((id, ex) -> {
-                if (ex == null) {
-                    deployFut.complete(id);
-                } else {
-                    deployFut.completeExceptionally(ex);
-                }
-            });
-
-            areWatchesDeployed = true;
-        } finally {
-            busyLock.leaveBusy();
-        }
+    public long appliedRevision() {
+        return appliedRevision;
     }
 
     @Override
-    public synchronized CompletableFuture<Long> registerExactWatch(ByteArray key, WatchListener lsnr) {
-        if (!busyLock.enterBusy()) {
-            return CompletableFuture.failedFuture(new NodeStoppingException());
-        }
-
-        try {
-            long watchId = watchAggregator.add(key, lsnr);
-
-            return updateWatches().thenApply(uuid -> watchId);
-        } finally {
-            busyLock.leaveBusy();
-        }
+    public void registerPrefixWatch(ByteArray key, WatchListener lsnr) {
+        storage.watchPrefix(key.bytes(), appliedRevision + 1, lsnr);
     }
 
-    /**
-     * Register watch listener by collection of keys.
-     *
-     * @param keys Collection listen.
-     * @param lsnr Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription
-     */
-    public synchronized CompletableFuture<Long> registerWatch(Collection<ByteArray> keys, WatchListener lsnr) {
-        if (!busyLock.enterBusy()) {
-            return CompletableFuture.failedFuture(new NodeStoppingException());
-        }
-
-        try {
-            long watchId = watchAggregator.add(keys, lsnr);
-
-            return updateWatches().thenApply(uuid -> watchId);
-        } finally {
-            busyLock.leaveBusy();
-        }
+    @Override
+    public void registerExactWatch(ByteArray key, WatchListener listener) {
+        storage.watchExact(key.bytes(), appliedRevision + 1, listener);
     }
 
     @Override
-    public synchronized CompletableFuture<Long> registerRangeWatch(ByteArray keyFrom, ByteArray keyTo, WatchListener lsnr) {
-        if (!busyLock.enterBusy()) {
-            return CompletableFuture.failedFuture(new NodeStoppingException());
-        }
-
-        try {
-            long watchId = watchAggregator.add(keyFrom, keyTo, lsnr);
-
-            return updateWatches().thenApply(uuid -> watchId);
-        } finally {
-            busyLock.leaveBusy();
-        }
+    public void registerRangeWatch(ByteArray keyFrom, @Nullable ByteArray keyTo, WatchListener listener) {
+        storage.watchRange(keyFrom.bytes(), keyTo == null ? null : keyTo.bytes(), appliedRevision + 1, listener);
     }
 
     @Override
-    public synchronized CompletableFuture<Long> registerPrefixWatch(ByteArray key, WatchListener lsnr) {
-        if (!busyLock.enterBusy()) {
-            return CompletableFuture.failedFuture(new NodeStoppingException());
-        }
-
-        try {
-            long watchId = watchAggregator.addPrefix(key, lsnr);
-
-            return updateWatches().thenApply(uuid -> watchId);
-        } finally {
-            busyLock.leaveBusy();
-        }
+    public void unregisterWatch(WatchListener lsnr) {
+        storage.removeWatch(lsnr);
     }
 
     @Override
-    public synchronized CompletableFuture<Void> unregisterWatch(long id) {
+    public void deployWatches() throws NodeStoppingException {
         if (!busyLock.enterBusy()) {
-            return CompletableFuture.failedFuture(new NodeStoppingException());
+            throw new NodeStoppingException();
         }
 
         try {
-            watchAggregator.cancel(id);
+            storage.startWatches(revision -> {
+                if (!busyLock.enterBusy()) {
+                    LOG.info("Skipping applying MetaStorage revision because the node is stopping");
 
-            return updateWatches().thenApply(uuid -> null);
+                    return;
+                }
+
+                try {
+                    saveRevision(revision);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            });
         } finally {
             busyLock.leaveBusy();
         }
@@ -594,8 +604,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
      *
      * @see MetaStorageService#range(ByteArray, ByteArray, long)
      */
-    public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound)
-            throws NodeStoppingException {
+    public Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound) throws NodeStoppingException {
         if (!busyLock.enterBusy()) {
             throw new NodeStoppingException();
         }
@@ -617,11 +626,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
      *
      * @see MetaStorageService#range(ByteArray, ByteArray, boolean)
      */
-    public @NotNull Cursor<Entry> range(
-            @NotNull ByteArray keyFrom,
-            @Nullable ByteArray keyTo,
-            boolean includeTombstones
-    ) throws NodeStoppingException {
+    public Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo, boolean includeTombstones) throws NodeStoppingException {
         if (!busyLock.enterBusy()) {
             throw new NodeStoppingException();
         }
@@ -652,42 +657,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         }
 
         try {
-            CompletableFuture<Cursor<Entry>> cursorFuture = metaStorageSvcFut.thenCombine(
-                    appliedRevision(),
-                    (svc, appliedRevision) -> svc.range(keyFrom, keyTo, appliedRevision)
-            );
-
-            return new CursorWrapper<>(cursorFuture);
-        } finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
-     * Retrieves entries for the given key prefix in lexicographic order. Entries will be filtered out by the current applied revision as an
-     * upper bound. Applied revision is a revision of the last successful vault update.
-     *
-     * <p>Prefix query is a synonym of the range query {@code (prefixKey, nextKey(prefixKey))}.
-     *
-     * @param keyPrefix Prefix of the key to retrieve the entries. Couldn't be {@code null}.
-     * @return Cursor built upon entries corresponding to the given range and applied revision.
-     * @throws OperationTimeoutException If the operation is timed out.
-     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
-     * @see ByteArray
-     * @see Entry
-     */
-    public Cursor<Entry> prefixWithAppliedRevision(ByteArray keyPrefix) throws NodeStoppingException {
-        if (!busyLock.enterBusy()) {
-            throw new NodeStoppingException();
-        }
-
-        try {
-            CompletableFuture<Cursor<Entry>> cursorFuture = metaStorageSvcFut.thenCombine(
-                    appliedRevision(),
-                    (svc, appliedRevision) -> svc.prefix(keyPrefix, appliedRevision)
-            );
-
-            return new CursorWrapper<>(cursorFuture);
+            return new CursorWrapper<>(metaStorageSvcFut.thenApply(svc -> svc.range(keyFrom, keyTo, appliedRevision)));
         } finally {
             busyLock.leaveBusy();
         }
@@ -729,69 +699,20 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
     }
 
     /**
-     * Returns applied revision for {@link VaultManager#putAll} operation.
+     * Returns applied revision from the local storage.
      */
-    private CompletableFuture<Long> appliedRevision() {
+    private CompletableFuture<Long> readAppliedRevision() {
         return vaultMgr.get(APPLIED_REV)
                 .thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()));
     }
 
     /**
-     * Stop current batch of consolidated watches and register new one from current {@link WatchAggregator}.
-     *
-     * <p>This method MUST always be called under a {@code synchronized} block.
-     *
-     * @return Ignite UUID of new consolidated watch.
-     */
-    private CompletableFuture<IgniteUuid> updateWatches() {
-        if (!areWatchesDeployed) {
-            return deployFut;
-        }
-
-        deployFut = deployFut
-                .thenCompose(id -> id == null
-                        ? CompletableFuture.completedFuture(null)
-                        : metaStorageSvcFut.thenCompose(svc -> svc.stopWatch(id))
-                )
-                .thenCompose(r -> updateAggregatedWatch());
-
-        return deployFut;
-    }
-
-    private CompletableFuture<IgniteUuid> updateAggregatedWatch() {
-        return appliedRevision()
-                .thenCompose(appliedRevision ->
-                        watchAggregator.watch(appliedRevision + 1, this::storeEntries)
-                                .map(this::dispatchAppropriateMetaStorageWatch)
-                                .orElseGet(() -> CompletableFuture.completedFuture(null))
-                );
-    }
-
-    /**
-     * Store entries with appropriate associated revision.
-     *
-     * @param entries to store.
-     * @param revision associated revision.
+     * Save processed Meta Storage revision.
      */
-    private void storeEntries(Collection<IgniteBiTuple<ByteArray, byte[]>> entries, long revision) {
-        appliedRevision()
-                .thenCompose(appliedRevision -> {
-                    if (revision <= appliedRevision) {
-                        throw new MetaStorageException(DEPLOYING_WATCH_ERR, String.format(
-                                "Current revision (%d) must be greater than the revision in the Vault (%d)",
-                                revision, appliedRevision
-                        ));
-                    }
+    private void saveRevision(long revision) {
+        vaultMgr.put(APPLIED_REV, longToBytes(revision)).join();
 
-                    Map<ByteArray, byte[]> batch = IgniteUtils.newHashMap(entries.size() + 1);
-
-                    batch.put(APPLIED_REV, longToBytes(revision));
-
-                    entries.forEach(e -> batch.put(e.getKey(), e.getValue()));
-
-                    return vaultMgr.putAll(batch);
-                })
-                .join();
+        appliedRevision = revision;
     }
 
     // TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
@@ -801,9 +722,6 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         /** Inner cursor future. */
         private final CompletableFuture<Cursor<T>> innerCursorFut;
 
-        /** Inner iterator future. */
-        private final CompletableFuture<Iterator<T>> innerIterFut;
-
         /**
          * Constructor.
          *
@@ -811,27 +729,12 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
          */
         CursorWrapper(CompletableFuture<Cursor<T>> innerCursorFut) {
             this.innerCursorFut = innerCursorFut;
-            this.innerIterFut = innerCursorFut.thenApply(Iterable::iterator);
         }
 
         /** {@inheritDoc} */
         @Override
         public void close() {
-            try {
-                innerCursorFut.thenAccept(cursor -> {
-                    try {
-                        cursor.close();
-                    } catch (RuntimeException e) {
-                        throw new MetaStorageException(CURSOR_CLOSING_ERR, e);
-                    }
-                }).get();
-            } catch (ExecutionException e) {
-                throw new IgniteInternalException("Exception while closing a cursor", e);
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-
-                throw new IgniteInternalException("Interrupted while closing a cursor", e);
-            }
+            get(innerCursorFut.thenAccept(Cursor::close), CURSOR_CLOSING_ERR);
         }
 
         /** {@inheritDoc} */
@@ -842,11 +745,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
             }
 
             try {
-                try {
-                    return innerIterFut.thenApply(Iterator::hasNext).get();
-                } catch (InterruptedException | ExecutionException e) {
-                    throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
-                }
+                return get(innerCursorFut.thenApply(Iterator::hasNext), CURSOR_EXECUTION_ERR);
             } finally {
                 busyLock.leaveBusy();
             }
@@ -860,48 +759,22 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
             }
 
             try {
-                try {
-                    return innerIterFut.thenApply(Iterator::next).get();
-                } catch (InterruptedException | ExecutionException e) {
-                    throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
-                }
+                return get(innerCursorFut.thenApply(Iterator::next), CURSOR_EXECUTION_ERR);
             } finally {
                 busyLock.leaveBusy();
             }
         }
-    }
 
-    /**
-     * Dispatches appropriate meta storage watch method according to inferred watch criterion.
-     *
-     * @param aggregatedWatch Aggregated watch.
-     * @return Future, which will be completed after new watch registration finished.
-     */
-    private CompletableFuture<IgniteUuid> dispatchAppropriateMetaStorageWatch(AggregatedWatch aggregatedWatch) {
-        if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.CollectionCriterion) {
-            var criterion = (KeyCriterion.CollectionCriterion) aggregatedWatch.keyCriterion();
-
-            return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
-                    criterion.keys(),
-                    aggregatedWatch.revision(),
-                    aggregatedWatch.listener()));
-        } else if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.ExactCriterion) {
-            var criterion = (KeyCriterion.ExactCriterion) aggregatedWatch.keyCriterion();
-
-            return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
-                    criterion.key(),
-                    aggregatedWatch.revision(),
-                    aggregatedWatch.listener()));
-        } else if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.RangeCriterion) {
-            var criterion = (KeyCriterion.RangeCriterion) aggregatedWatch.keyCriterion();
-
-            return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
-                    criterion.from(),
-                    criterion.to(),
-                    aggregatedWatch.revision(),
-                    aggregatedWatch.listener()));
-        } else {
-            throw new UnsupportedOperationException("Unsupported type of criterion");
+        private <R> R get(CompletableFuture<R> future, int errorCode) {
+            try {
+                return future.get();
+            } catch (ExecutionException e) {
+                throw new MetaStorageException(errorCode, e);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new MetaStorageException(errorCode, e);
+            }
         }
     }
 }
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
index 9d4f133066..b774971013 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
@@ -22,7 +22,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Condition;
 import org.apache.ignite.internal.metastorage.dsl.If;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
@@ -31,7 +30,6 @@ import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
 import org.apache.ignite.internal.metastorage.exceptions.OperationTimeoutException;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -52,12 +50,12 @@ public interface MetaStorageService {
     /**
      * Retrieves an entry for the given key and the revision upper bound.
      *
-     * @param key           The key. Couldn't be {@code null}.
+     * @param key The key. Couldn't be {@code null}.
      * @param revUpperBound The upper bound for entry revisions. Must be positive.
      * @return An entry for the given key and maximum revision limited by {@code revUpperBound}. Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction. Will be thrown on
-     *                                   getting future result.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on getting
+     *      future result.
      * @see ByteArray
      * @see Entry
      */
@@ -77,12 +75,12 @@ public interface MetaStorageService {
     /**
      * Retrieves entries for given keys and the revision upper bound.
      *
-     * @param keys          The set of keys. Couldn't be {@code null} or empty. Set elements couldn't be {@code null}.
+     * @param keys The set of keys. Couldn't be {@code null} or empty. Set elements couldn't be {@code null}.
      * @param revUpperBound The upper bound for entry revisions. Must be positive.
      * @return A map of entries for given keys and maximum revision limited by {@code revUpperBound}. Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction. Will be thrown on
-     *                                   getting future result.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on getting
+     *      future result.
      * @see ByteArray
      * @see Entry
      */
@@ -91,7 +89,7 @@ public interface MetaStorageService {
     /**
      * Inserts or updates an entry with the given key and the given value.
      *
-     * @param key   The key. Couldn't be {@code null}.
+     * @param key The key. Couldn't be {@code null}.
      * @param value The value. Couldn't be {@code null}.
      * @return Completed future.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
@@ -103,7 +101,7 @@ public interface MetaStorageService {
     /**
      * Inserts or updates an entry with the given key and the given value and retrieves a previous entry for the given key.
      *
-     * @param key   The key. Couldn't be {@code null}.
+     * @param key The key. Couldn't be {@code null}.
      * @param value The value. Couldn't be {@code null}.
      * @return A previous entry for the given key. Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
@@ -185,8 +183,8 @@ public interface MetaStorageService {
      * <p>Conditional update could be treated as <i>if(condition)-then(success)-else(failure)</i> expression.</p>
      *
      * @param condition The condition.
-     * @param success   The update which will be applied in case of condition evaluation yields {@code true}.
-     * @param failure   The update which will be applied in case of condition evaluation yields {@code false}.
+     * @param success The update which will be applied in case of condition evaluation yields {@code true}.
+     * @param failure The update which will be applied in case of condition evaluation yields {@code false}.
      * @return Future result {@code true} if {@code success} update was applied, otherwise {@code false}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
      * @see ByteArray
@@ -202,8 +200,8 @@ public interface MetaStorageService {
      * <p>Conditional update could be treated as <i>if(condition)-then(success)-else(failure)</i> expression.</p>
      *
      * @param condition The condition.
-     * @param success   The updates which will be applied in case of condition evaluation yields {@code true}.
-     * @param failure   The updates which will be applied in case of condition evaluation yields {@code false}.
+     * @param success The updates which will be applied in case of condition evaluation yields {@code true}.
+     * @param failure The updates which will be applied in case of condition evaluation yields {@code false}.
      * @return Future result {@code true} if {@code success} update was applied, otherwise {@code false}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
      * @see ByteArray
@@ -214,12 +212,11 @@ public interface MetaStorageService {
     CompletableFuture<Boolean> invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure);
 
     /**
-     * Invoke, which supports nested conditional statements.
-     * For detailed docs about construction of new if statement, look at {@link If} javadocs.
+     * Invoke, which supports nested conditional statements. For detailed docs about construction of new if statement, look at {@link If}
+     * javadocs.
      *
      * @param iif {@link If} statement to invoke
      * @return execution result
-     *
      * @see If
      * @see StatementResult
      */
@@ -229,12 +226,12 @@ public interface MetaStorageService {
      * Retrieves entries for the given key range in lexicographic order. Entries will be filtered out by upper bound of given revision
      * number.
      *
-     * @param keyFrom       Start key of range (inclusive). Couldn't be {@code null}.
-     * @param keyTo         End key of range (exclusive). Could be {@code null}.
+     * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+     * @param keyTo End key of range (exclusive). Could be {@code null}.
      * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
      * @return Cursor built upon entries corresponding to the given range and revision.
      * @throws OperationTimeoutException If the operation is timed out.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
      * @see ByteArray
      * @see Entry
      */
@@ -244,13 +241,13 @@ public interface MetaStorageService {
      * Retrieves entries for the given key range in lexicographic order. Entries will be filtered out by upper bound of given revision
      * number.
      *
-     * @param keyFrom           Start key of range (inclusive). Couldn't be {@code null}.
-     * @param keyTo             End key of range (exclusive). Could be {@code null}.
-     * @param revUpperBound     The upper bound for entry revision. {@code -1} means latest revision.
+     * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+     * @param keyTo End key of range (exclusive). Could be {@code null}.
+     * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
      * @param includeTombstones Whether to include tombstone entries.
      * @return Cursor built upon entries corresponding to the given range and revision.
      * @throws OperationTimeoutException If the operation is timed out.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
      * @see ByteArray
      * @see Entry
      */
@@ -261,25 +258,25 @@ public interface MetaStorageService {
      * {@code revUpperBound == -1}.
      *
      * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
-     * @param keyTo   End key of range (exclusive). Could be {@code null}.
+     * @param keyTo End key of range (exclusive). Could be {@code null}.
      * @return Cursor built upon entries corresponding to the given range and revision.
      * @throws OperationTimeoutException If the operation is timed out.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
      * @see ByteArray
      * @see Entry
      */
     Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo);
 
     /**
-     * Retrieves entries for the given key range in lexicographic order. Short cut for
-     * {@link #range(ByteArray, ByteArray, long, boolean)} where {@code revUpperBound == -1}.
+     * Retrieves entries for the given key range in lexicographic order. Short cut for {@link #range(ByteArray, ByteArray, long, boolean)}
+     * where {@code revUpperBound == -1}.
      *
-     * @param keyFrom           Start key of range (inclusive). Couldn't be {@code null}.
-     * @param keyTo             End key of range (exclusive). Could be {@code null}.
+     * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+     * @param keyTo End key of range (exclusive). Could be {@code null}.
      * @param includeTombstones Whether to include tombstone entries.
      * @return Cursor built upon entries corresponding to the given range and revision.
      * @throws OperationTimeoutException If the operation is timed out.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction.
+     * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
      * @see ByteArray
      * @see Entry
      */
@@ -296,64 +293,6 @@ public interface MetaStorageService {
      */
     Cursor<Entry> prefix(ByteArray prefix, long revUpperBound);
 
-    /**
-     * Subscribes on meta storage updates matching the parameters.
-     *
-     * @param keyFrom  Start key of range (inclusive). Could be {@code null}.
-     * @param keyTo    End key of range (exclusive). Could be {@code null}.
-     * @param revision Start revision inclusive. {@code 0} - all revisions, {@code -1} - latest revision (accordingly to current meta
-     *                 storage state).
-     * @param lsnr     Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
-     * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction. Will be thrown on
-     *                                   getting future result.
-     * @see ByteArray
-     * @see Entry
-     */
-    CompletableFuture<IgniteUuid> watch(@Nullable ByteArray keyFrom, @Nullable ByteArray keyTo, long revision, WatchListener lsnr);
-
-    /**
-     * Subscribes on meta storage updates for the given key.
-     *
-     * @param key      The target key. Couldn't be {@code null}.
-     * @param revision Start revision inclusive. {@code 0} - all revisions, {@code -1} - latest revision (accordingly to current meta
-     *                 storage state).
-     * @param lsnr     Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
-     * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction. Will be thrown on
-     *                                   getting future result.
-     * @see ByteArray
-     * @see Entry
-     */
-    CompletableFuture<IgniteUuid> watch(ByteArray key, long revision, WatchListener lsnr);
-
-    /**
-     * Subscribes on meta storage updates for given keys.
-     *
-     * @param keys     Set of target keys. Couldn't be {@code null} or empty.
-     * @param revision Start revision inclusive. {@code 0} - all revision, {@code -1} - latest revision (accordingly to current meta storage
-     *                 state).
-     * @param lsnr     Listener which will be notified for each update.
-     * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
-     * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @throws CompactedException        If the desired revisions are removed from the storage due to a compaction. Will be thrown on
-     *                                   getting future result.
-     * @see ByteArray
-     * @see Entry
-     */
-    CompletableFuture<IgniteUuid> watch(Set<ByteArray> keys, long revision, WatchListener lsnr);
-
-    /**
-     * Cancels subscription for the given identifier.
-     *
-     * @param id Subscription identifier.
-     * @return Completed future in case of operation success. Couldn't be {@code null}.
-     * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     */
-    CompletableFuture<Void> stopWatch(IgniteUuid id);
-
     /**
      * Compacts meta storage (removes all tombstone entries and old entries except of entries with latest revision).
      *
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
index 52e8611247..a1bbe42ca1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
@@ -23,26 +23,16 @@ import static org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand
 import static org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand.getAndRemoveAllCommand;
 import static org.apache.ignite.internal.metastorage.command.PutAllCommand.putAllCommand;
 import static org.apache.ignite.internal.metastorage.command.RemoveAllCommand.removeAllCommand;
-import static org.apache.ignite.internal.metastorage.command.WatchExactKeysCommand.watchExactKeysCommand;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_STOPPING_ERR;
 
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.Iterator;
 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.ConcurrentHashMap;
-import java.util.concurrent.RejectedExecutionException;
-import org.apache.ignite.internal.logger.IgniteLogger;
-import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.command.GetAllCommand;
 import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
 import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
@@ -79,23 +69,17 @@ import org.apache.ignite.internal.metastorage.dsl.SimpleCondition.ValueCondition
 import org.apache.ignite.internal.metastorage.dsl.Statement;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
 import org.apache.ignite.internal.metastorage.dsl.Update;
-import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.IgniteUuidGenerator;
-import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * {@link MetaStorageService} implementation.
  */
 public class MetaStorageServiceImpl implements MetaStorageService {
-    /** The logger. */
-    private static final IgniteLogger LOG = Loggers.forClass(MetaStorageServiceImpl.class);
-
     /** IgniteUuid generator. */
     private static final IgniteUuidGenerator uuidGenerator = new IgniteUuidGenerator(UUID.randomUUID(), 0);
 
@@ -105,28 +89,22 @@ public class MetaStorageServiceImpl implements MetaStorageService {
     /** Meta storage raft group service. */
     private final RaftGroupService metaStorageRaftGrpSvc;
 
-    // TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
-    /** Watch processor, that uses pulling logic in order to retrieve watch notifications from server. */
-    private final WatchProcessor watchProcessor;
-
-    /** Local node id. */
-    private final String localNodeId;
-
-    /** Local node name. */
-    private final String localNodeName;
+    /** Local node. */
+    private final ClusterNode localNode;
 
     /**
      * Constructor.
      *
      * @param metaStorageRaftGrpSvc Meta storage raft group service.
-     * @param localNodeId Local node id.
-     * @param localNodeName Local node name.
+     * @param localNode Local node.
      */
-    public MetaStorageServiceImpl(RaftGroupService metaStorageRaftGrpSvc, String localNodeId, String localNodeName) {
+    public MetaStorageServiceImpl(RaftGroupService metaStorageRaftGrpSvc, ClusterNode localNode) {
         this.metaStorageRaftGrpSvc = metaStorageRaftGrpSvc;
-        this.watchProcessor = new WatchProcessor();
-        this.localNodeId = localNodeId;
-        this.localNodeName = localNodeName;
+        this.localNode = localNode;
+    }
+
+    RaftGroupService raftGroupService() {
+        return metaStorageRaftGrpSvc;
     }
 
     /** {@inheritDoc} */
@@ -282,7 +260,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
                         commandsFactory.rangeCommand()
                                 .keyFrom(keyFrom.bytes())
                                 .keyTo(keyTo == null ? null : keyTo.bytes())
-                                .requesterNodeId(localNodeId)
+                                .requesterNodeId(localNode.id())
                                 .cursorId(uuidGenerator.randomUuid())
                                 .revUpperBound(revUpperBound)
                                 .includeTombstones(includeTombstones)
@@ -314,7 +292,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
                         commandsFactory.prefixCommand()
                                 .prefix(prefix.bytes())
                                 .revUpperBound(revUpperBound)
-                                .requesterNodeId(localNodeId)
+                                .requesterNodeId(localNode.id())
                                 .cursorId(uuidGenerator.randomUuid())
                                 .includeTombstones(false)
                                 .batchSize(PrefixCommand.DEFAULT_BATCH_SIZE)
@@ -324,74 +302,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         );
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<IgniteUuid> watch(
-            @Nullable ByteArray keyFrom,
-            @Nullable ByteArray keyTo,
-            long revision,
-            WatchListener lsnr
-    ) {
-        CompletableFuture<IgniteUuid> watchRes = metaStorageRaftGrpSvc.run(commandsFactory.watchRangeKeysCommand()
-                .keyFrom(keyFrom == null ? null : keyFrom.bytes())
-                .keyTo(keyTo == null ? null : keyTo.bytes())
-                .revision(revision)
-                .requesterNodeId(localNodeId)
-                .cursorId(uuidGenerator.randomUuid())
-                .build()
-        );
-
-        watchRes.thenAccept(
-                watchId -> watchProcessor.addWatch(
-                        watchId,
-                        new CursorImpl<>(commandsFactory, metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
-                        lsnr
-                )
-        );
-
-        return watchRes;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<IgniteUuid> watch(
-            ByteArray key,
-            long revision,
-            WatchListener lsnr
-    ) {
-        return watch(key, null, revision, lsnr);
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<IgniteUuid> watch(
-            Set<ByteArray> keys,
-            long revision,
-            WatchListener lsnr
-    ) {
-        CompletableFuture<IgniteUuid> watchRes =
-                metaStorageRaftGrpSvc.run(watchExactKeysCommand(commandsFactory, keys, revision, localNodeId, uuidGenerator.randomUuid()));
-
-        watchRes.thenAccept(
-                watchId -> watchProcessor.addWatch(
-                        watchId,
-                        new CursorImpl<>(commandsFactory, metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
-                        lsnr
-                )
-        );
-
-        return watchRes;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> stopWatch(IgniteUuid id) {
-        return CompletableFuture.runAsync(() -> watchProcessor.stopWatch(id));
-    }
-
     // TODO: IGNITE-14734 Implement.
-
-    /** {@inheritDoc} */
     @Override
     public CompletableFuture<Void> compact() {
         throw new UnsupportedOperationException();
@@ -489,9 +400,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
                     .conditionType(cond.compoundConditionType().ordinal())
                     .build();
         } else {
-            assert false : "Unknown condition type: " + condition.getClass().getSimpleName();
-
-            return null;
+            throw new IllegalArgumentException("Unknown condition type: " + condition.getClass().getSimpleName());
         }
     }
 
@@ -520,153 +429,4 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
         return new EntryImpl(resp.key(), resp.value(), resp.revision(), resp.updateCounter());
     }
-
-    private static WatchEvent watchResponse(Object obj) {
-        MultipleEntryResponse resp = (MultipleEntryResponse) obj;
-
-        List<EntryEvent> evts = new ArrayList<>(resp.entries().size() / 2);
-
-        Entry o = null;
-        Entry n;
-
-        for (int i = 0; i < resp.entries().size(); i++) {
-            SingleEntryResponse s = resp.entries().get(i);
-
-            EntryImpl e = new EntryImpl(s.key(), s.value(), s.revision(), s.updateCounter());
-
-            if (i % 2 == 0) {
-                o = e;
-            } else {
-                n = e;
-
-                evts.add(new EntryEvent(o, n));
-            }
-        }
-
-        return new WatchEvent(evts);
-    }
-
-    // TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
-
-    /** Watch processor, that manages {@link Watcher} threads. */
-    private final class WatchProcessor {
-        /** Active Watcher threads that process notification pulling logic. */
-        private final Map<IgniteUuid, Watcher> watchers = new ConcurrentHashMap<>();
-
-        /**
-         * Starts exclusive thread per watch that implement watch pulling logic and calls {@link WatchListener#onUpdate(WatchEvent)}} or
-         * {@link WatchListener#onError(Throwable)}.
-         *
-         * @param watchId Watch id.
-         * @param cursor Watch Cursor.
-         * @param lsnr The listener which receives and handles watch updates.
-         */
-        private void addWatch(IgniteUuid watchId, CursorImpl<WatchEvent> cursor, WatchListener lsnr) {
-            Watcher watcher = new Watcher(cursor, lsnr);
-
-            watchers.put(watchId, watcher);
-
-            watcher.start();
-        }
-
-        /**
-         * Closes server cursor and interrupts watch pulling thread.
-         *
-         * @param watchId Watch id.
-         */
-        private void stopWatch(IgniteUuid watchId) {
-            watchers.computeIfPresent(
-                    watchId,
-                    (k, v) -> {
-                        CompletableFuture.runAsync(() -> {
-                            v.stop = true;
-
-                            v.interrupt();
-                        }).thenRun(() -> {
-                            try {
-                                Thread.sleep(100);
-
-                                v.cursor.close();
-                            } catch (InterruptedException e) {
-                                throw new MetaStorageException(WATCH_STOPPING_ERR, e);
-                            } catch (Exception e) {
-                                if (e instanceof IgniteInternalException && e.getCause().getCause() instanceof RejectedExecutionException) {
-                                    LOG.debug("Cursor close command was rejected because raft executor has been already stopped");
-                                    return;
-                                }
-
-                                // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
-                                LOG.warn("Unexpected exception", e);
-                            }
-                        });
-                        return null;
-                    }
-            );
-        }
-
-        /** Watcher thread, uses pulling logic in order to retrieve watch notifications from server. */
-        private final class Watcher extends Thread {
-            private volatile boolean stop = false;
-
-            /** Watch event cursor. */
-            private Cursor<WatchEvent> cursor;
-
-            /** The listener which receives and handles watch updates. */
-            private WatchListener lsnr;
-
-            /**
-             * Constructor.
-             *
-             * @param cursor Watch event cursor.
-             * @param lsnr The listener which receives and handles watch updates.
-             */
-            Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
-                setName("ms-watcher-" + localNodeName);
-                this.cursor = cursor;
-                this.lsnr = lsnr;
-            }
-
-            /**
-             * Pulls watch events from server side with the help of cursor.iterator.hasNext()/next() in the while(true) loop. Collects watch
-             * events with same revision and fires either onUpdate or onError().
-             */
-            @Override
-            public void run() {
-                Iterator<WatchEvent> watchEvtsIter = cursor.iterator();
-
-                while (!stop) {
-                    try {
-                        if (watchEvtsIter.hasNext()) {
-                            WatchEvent watchEvt = null;
-
-                            try {
-                                watchEvt = watchEvtsIter.next();
-                            } catch (Throwable e) {
-                                lsnr.onError(e);
-
-                                throw e;
-                            }
-
-                            assert watchEvt != null;
-
-                            lsnr.onUpdate(watchEvt);
-                        } else {
-                            Thread.sleep(10);
-                        }
-                    } catch (Throwable e) {
-                        if (e instanceof NodeStoppingException || e.getCause() instanceof NodeStoppingException) {
-                            break;
-                        } else if ((e instanceof InterruptedException || e.getCause() instanceof InterruptedException) && stop) {
-                            LOG.debug("Watcher has been stopped during node's stop");
-
-                            break;
-                        } else {
-                            // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
-                            LOG.warn("Unexpected exception", e);
-                        }
-                    }
-                }
-            }
-        }
-    }
 }
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
index 33adcb47c1..38254c6dd1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
@@ -21,11 +21,13 @@ import java.nio.file.Path;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.function.LongConsumer;
 import org.apache.ignite.internal.close.ManuallyCloseable;
 import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
+import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
 import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.Nullable;
 
@@ -220,32 +222,54 @@ public interface KeyValueStorage extends ManuallyCloseable {
      * @param keyFrom Start key of range (inclusive).
      * @param keyTo   Last key of range (exclusive).
      * @param rev     Start revision number.
-     * @return Cursor by update events.
      */
-    Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev);
+    void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener);
 
     /**
-     * Creates subscription on updates of entries corresponding to the given keys range (where upper bound is unlimited) and starting from
-     * the given revision number.
+     * Registers a watch listener by a key prefix.
      *
-     * @param key Start key of range (inclusive).
-     * @param rev Start revision number.
-     * @return Cursor by update events.
+     * @param prefix Prefix to listen to.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
      */
-    Cursor<WatchEvent> watch(byte[] key, long rev);
+    void watchPrefix(byte[] prefix, long rev, WatchListener listener);
 
     /**
-     * Creates subscription on updates of entries corresponding to the given keys collection and starting from the given revision number.
+     * Registers a watch listener for the provided key.
      *
-     * @param keys Collection of keys
-     * @param rev  Start revision number.
-     * @return Cursor by update events.
+     * @param key Meta Storage key.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
      */
-    Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
+    void watchExact(byte[] key, long rev, WatchListener listener);
+
+    /**
+     * Registers a watch listener for the provided keys.
+     *
+     * @param keys Meta Storage keys.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
+     */
+    void watchExact(Collection<byte[]> keys, long rev, WatchListener listener);
+
+    /**
+     * Starts all registered watches.
+     *
+     * <p>Before calling this method, watches will not receive any updates.
+     *
+     * @param revisionCallback Callback that will be invoked after all watches of a particular revision are processed, with the revision
+     *      as its argument.
+     */
+    void startWatches(LongConsumer revisionCallback);
+
+    /**
+     * Unregisters a watch listener.
+     */
+    void removeWatch(WatchListener listener);
 
     /**
      * Compacts storage (removes tombstones).
-     * TODO: IGNITE-16444 Сorrect compaction for Metastorage.
+     * TODO: IGNITE-16444 Correct compaction for Metastorage.
      */
     void compact();
 
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
new file mode 100644
index 0000000000..489de7c898
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
@@ -0,0 +1,89 @@
+/*
+ * 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.metastorage.server;
+
+import java.util.function.Predicate;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+
+/**
+ * Subscription on updates of Meta Storage entries corresponding to a subset of keys, starting from a given revision number.
+ */
+public class Watch {
+    /** Current revision. */
+    private volatile long targetRevision;
+
+    /** Key predicate. */
+    private final Predicate<byte[]> predicate;
+
+    /** Event listener. */
+    private final WatchListener listener;
+
+    /**
+     * Constructor.
+     *
+     * @param startRevision Starting revision.
+     * @param listener Event listener.
+     * @param predicate Key predicate.
+     */
+    public Watch(long startRevision, WatchListener listener, Predicate<byte[]> predicate) {
+        this.predicate = predicate;
+        this.listener = listener;
+        this.targetRevision = startRevision;
+    }
+
+    /**
+     * Returns {@code true} if a given key and its revision should be forwarded to the event listener.
+     *
+     * @param key Meta Storage key.
+     * @param revision Revision corresponding to the given {@code key}.
+     */
+    public boolean matches(byte[] key, long revision) {
+        return revision >= targetRevision && predicate.test(key);
+    }
+
+    /**
+     * Notifies the event listener about a Meta Storage event.
+     */
+    public void onUpdate(WatchEvent event) {
+        listener.onUpdate(event);
+
+        targetRevision = event.revision() + 1;
+    }
+
+    /**
+     * Callback that gets called if an error has occurred during the event processing.
+     */
+    public void onError(Throwable e) {
+        listener.onError(e);
+    }
+
+    /**
+     * Returns the event listener.
+     */
+    public WatchListener listener() {
+        return listener;
+    }
+
+    /**
+     * Returns the current Meta Storage revision this Watch is listening to.
+     */
+    public long targetRevision() {
+        return targetRevision;
+    }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
index 6b321a170d..77ad1cffac 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
@@ -24,6 +24,8 @@ import static org.apache.ignite.internal.metastorage.server.persistence.RocksSto
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.getAsLongs;
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.keyToRocksKey;
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.longToBytes;
+import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.revisionFromRocksKey;
+import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.rocksKeyToBytes;
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.valueToBytes;
 import static org.apache.ignite.internal.metastorage.server.persistence.StorageColumnFamilyType.DATA;
 import static org.apache.ignite.internal.metastorage.server.persistence.StorageColumnFamilyType.INDEX;
@@ -34,6 +36,7 @@ import static org.apache.ignite.lang.ErrorGroups.MetaStorage.COMPACTION_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.RESTORING_STORAGE_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.STARTING_STORAGE_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_EXECUTION_ERR;
 
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
@@ -44,14 +47,25 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.LongConsumer;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
 import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
 import org.apache.ignite.internal.metastorage.dsl.Update;
@@ -62,10 +76,12 @@ import org.apache.ignite.internal.metastorage.server.If;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
 import org.apache.ignite.internal.metastorage.server.Statement;
 import org.apache.ignite.internal.metastorage.server.Value;
+import org.apache.ignite.internal.metastorage.server.Watch;
 import org.apache.ignite.internal.rocksdb.ColumnFamily;
 import org.apache.ignite.internal.rocksdb.RocksBiPredicate;
 import org.apache.ignite.internal.rocksdb.RocksUtils;
 import org.apache.ignite.internal.rocksdb.snapshot.RocksSnapshotManager;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -81,6 +97,7 @@ import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.RocksIterator;
+import org.rocksdb.Slice;
 import org.rocksdb.WriteBatch;
 import org.rocksdb.WriteOptions;
 
@@ -96,6 +113,8 @@ import org.rocksdb.WriteOptions;
  * entry and the value is a {@code byte[]} that represents a {@code long[]} where every item is a revision of the storage.
  */
 public class RocksDbKeyValueStorage implements KeyValueStorage {
+    private static final IgniteLogger LOG = Loggers.forClass(RocksDbKeyValueStorage.class);
+
     /** A revision to store with system entries. */
     private static final long SYSTEM_REVISION_MARKER_VALUE = 0;
 
@@ -127,7 +146,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
 
     /** Thread-pool for snapshot operations execution. */
-    private final ExecutorService snapshotExecutor = Executors.newFixedThreadPool(2);
+    private final ExecutorService snapshotExecutor;
 
     /** Path to the rocksdb database. */
     private final Path dbPath;
@@ -153,13 +172,55 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     /** Update counter. Will be incremented for each update of any particular entry. */
     private volatile long updCntr;
 
+    /** Executor for processing watch events. */
+    private final ExecutorService watchExecutor;
+
+    /**
+     * Future representing the task of polling the {@code eventQueue}. Needed for cancelling the task on storage stop.
+     */
+    @Nullable
+    private volatile Future<?> watchExecutorFuture;
+
+    /** List of watches that listen to storage events. */
+    private final List<Watch> watches = new CopyOnWriteArrayList<>();
+
+    /** Status of the watch recovery process. */
+    private enum RecoveryStatus {
+        INITIAL,
+        PENDING,
+        IN_PROGRESS,
+        DONE
+    }
+
+    /**
+     * Current status of the watch recovery process. Watch recovery is needed for replaying missed updated when {@link #startWatches}
+     * is called.
+     */
+    private final AtomicReference<RecoveryStatus> recoveryStatus = new AtomicReference<>(RecoveryStatus.INITIAL);
+
+    /** Queue of update events, consumed by the {@link #watchExecutor}. */
+    private final BlockingQueue<List<Entry>> eventQueue = new LinkedBlockingQueue<>();
+
+    /**
+     * Current list of updated entries.
+     *
+     * <p>Since this list gets read and updated only on writes (under a write lock), no extra synchronisation is needed.
+     */
+    private final List<Entry> updatedEntries = new ArrayList<>();
+
+    /** Callback that gets notified after a {@link WatchEvent} has been processed by all registered watches. */
+    private volatile LongConsumer revisionCallback;
+
     /**
      * Constructor.
      *
      * @param dbPath RocksDB path.
      */
-    public RocksDbKeyValueStorage(Path dbPath) {
+    public RocksDbKeyValueStorage(String nodeName, Path dbPath) {
         this.dbPath = dbPath;
+
+        this.snapshotExecutor = Executors.newFixedThreadPool(2, NamedThreadFactory.create(nodeName, "metastorage-snapshot-executor", LOG));
+        this.watchExecutor = Executors.newSingleThreadExecutor(NamedThreadFactory.create(nodeName, "metastorage-watch-executor", LOG));
     }
 
     /** {@inheritDoc} */
@@ -228,6 +289,14 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     /** {@inheritDoc} */
     @Override
     public void close() {
+        Future<?> f = watchExecutorFuture;
+
+        if (f != null) {
+            f.cancel(true);
+        }
+
+        IgniteUtils.shutdownAndAwaitTermination(watchExecutor, 10, TimeUnit.SECONDS);
+
         IgniteUtils.shutdownAndAwaitTermination(snapshotExecutor, 10, TimeUnit.SECONDS);
 
         RocksUtils.closeAll(db, options);
@@ -242,6 +311,8 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     /** {@inheritDoc} */
     @Override
     public void restoreSnapshot(Path path) {
+        long currentRevision;
+
         rwLock.writeLock().lock();
 
         try {
@@ -252,7 +323,9 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             snapshotManager.restoreSnapshot(path);
 
-            rev = bytesToLong(data.get(REVISION_KEY));
+            currentRevision = bytesToLong(data.get(REVISION_KEY));
+
+            rev = currentRevision;
 
             updCntr = bytesToLong(data.get(UPDATE_COUNTER_KEY));
         } catch (Exception e) {
@@ -260,6 +333,11 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
         } finally {
             rwLock.writeLock().unlock();
         }
+
+        // Replay updates if startWatches() has already been called.
+        if (recoveryStatus.compareAndSet(RecoveryStatus.PENDING, RecoveryStatus.IN_PROGRESS)) {
+            replayUpdates(currentRevision);
+        }
     }
 
     /** {@inheritDoc} */
@@ -333,6 +411,14 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
             rev = newRev;
             updCntr = newCntr;
         }
+
+        queueWatchEvent();
+    }
+
+    private static Entry entry(byte[] key, long revision, Value value) {
+        return value.tombstone()
+                ? EntryImpl.tombstone(key, revision, value.updateCounter())
+                : new EntryImpl(key, value.bytes(), revision, value.updateCounter());
     }
 
     /** {@inheritDoc} */
@@ -347,7 +433,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             long[] revs = getRevisions(key);
 
-            final long lastRev = revs.length == 0 ? 0 : lastRevision(revs);
+            long lastRev = revs.length == 0 ? 0 : lastRevision(revs);
 
             addDataToBatch(batch, key, value, curRev, cntr);
 
@@ -710,25 +796,34 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     }
 
     @Override
-    public Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev) {
+    public void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener) {
         assert keyFrom != null : "keyFrom couldn't be null.";
         assert rev > 0 : "rev must be positive.";
 
-        return new WatchCursor(this, rev, k ->
-                CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
-        );
+        Predicate<byte[]> rangePredicate = keyTo == null
+                ? k -> CMP.compare(keyFrom, k) <= 0
+                : k -> CMP.compare(keyFrom, k) <= 0 && CMP.compare(keyTo, k) > 0;
+
+        watches.add(new Watch(rev, listener, rangePredicate));
+    }
+
+    @Override
+    public void watchPrefix(byte[] prefix, long rev, WatchListener listener) {
+        watchRange(prefix, incrementArray(prefix), rev, listener);
     }
 
     @Override
-    public Cursor<WatchEvent> watch(byte[] key, long rev) {
+    public void watchExact(byte[] key, long rev, WatchListener listener) {
         assert key != null : "key couldn't be null.";
         assert rev > 0 : "rev must be positive.";
 
-        return new WatchCursor(this, rev, k -> CMP.compare(k, key) == 0);
+        Predicate<byte[]> exactPredicate = k -> CMP.compare(k, key) == 0;
+
+        watches.add(new Watch(rev, listener, exactPredicate));
     }
 
     @Override
-    public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+    public void watchExact(Collection<byte[]> keys, long rev, WatchListener listener) {
         assert keys != null && !keys.isEmpty() : "keys couldn't be null or empty: " + keys;
         assert rev > 0 : "rev must be positive.";
 
@@ -736,7 +831,46 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
         keySet.addAll(keys);
 
-        return new WatchCursor(this, rev, keySet::contains);
+        Predicate<byte[]> inPredicate = keySet::contains;
+
+        watches.add(new Watch(rev, listener, inPredicate));
+    }
+
+    @Override
+    public void startWatches(LongConsumer revisionCallback) {
+        long currentRevision;
+
+        rwLock.readLock().lock();
+
+        try {
+            assert this.revisionCallback == null;
+
+            this.revisionCallback = revisionCallback;
+
+            currentRevision = rev;
+
+            // We update the recovery status under the read lock in order to avoid races between starting watches and applying a snapshot
+            // or concurrent writes. Replay of events can be done outside of the read lock relying on RocksDB snapshot isolation.
+            if (currentRevision == 0) {
+                // Revision can be 0 if there's no data in the storage. We set the status to PENDING expecting that it will be further
+                // updated either by applying a snapshot or by the first write to the storage.
+                recoveryStatus.set(RecoveryStatus.PENDING);
+            } else {
+                // If revision is not 0, we need to replay updates that match the existing data.
+                recoveryStatus.set(RecoveryStatus.IN_PROGRESS);
+            }
+        } finally {
+            rwLock.readLock().unlock();
+        }
+
+        if (currentRevision != 0) {
+            replayUpdates(currentRevision);
+        }
+    }
+
+    @Override
+    public void removeWatch(WatchListener listener) {
+        watches.removeIf(watch -> watch.listener() == listener);
     }
 
     @Override
@@ -815,7 +949,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
      * @param rev  Target revision.
      * @return Collection of entries.
      */
-    @NotNull
     private Collection<Entry> doGetAll(Collection<byte[]> keys, long rev) {
         assert keys != null : "keys list can't be null.";
         assert !keys.isEmpty() : "keys list can't be empty.";
@@ -843,8 +976,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
      * @param revUpperBound  Target upper bound of revision.
      * @return Value.
      */
-    @NotNull
-    Entry doGet(byte[] key, long revUpperBound) {
+    private Entry doGet(byte[] key, long revUpperBound) {
         assert revUpperBound >= LATEST_REV : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
 
         long[] revs;
@@ -878,7 +1010,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
      * Get a list of the revisions of the entry corresponding to the key.
      *
      * @param key Key.
-     * @return List of the revisions.
+     * @return Array of revisions.
      * @throws RocksDBException If failed to perform {@link RocksDB#get(ColumnFamilyHandle, byte[])}.
      */
     private long[] getRevisions(byte[] key) throws RocksDBException {
@@ -918,7 +1050,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
      * @param revision Target revision.
      * @return Entry.
      */
-    @NotNull
     Entry doGetValue(byte[] key, long revision) {
         if (revision == 0) {
             return EntryImpl.empty(key);
@@ -961,6 +1092,8 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
         byte[] rocksValue = valueToBytes(value, cntr);
 
         data.put(batch, rocksKey, rocksValue);
+
+        updatedEntries.add(entry(key, curRev, new Value(value, cntr)));
     }
 
     /**
@@ -1039,16 +1172,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
         }
     }
 
-    /**
-     * Creates a new iterator over the {@link StorageColumnFamilyType#DATA} column family.
-     *
-     * @param options Read options.
-     * @return Iterator.
-     */
-    public RocksIterator newDataIterator(ReadOptions options) {
-        return data.newIterator(options);
-    }
-
     /**
      * Gets last revision from the list.
      *
@@ -1060,17 +1183,166 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     }
 
     /**
-     * Returns database lock.
+     * Adds modified entries to the watch event queue.
      */
-    ReadWriteLock lock() {
-        return rwLock;
+    private void queueWatchEvent() {
+        if (updatedEntries.isEmpty()) {
+            return;
+        }
+
+        switch (recoveryStatus.get()) {
+            case INITIAL:
+                // Watches haven't been enabled yet, no need to queue any events, they will be replayed upon recovery.
+                updatedEntries.clear();
+
+                break;
+
+            case PENDING:
+                // Watches have been enabled, but no event replay happened because there was no data in the
+                // storage. Since we are adding some data here, we need to start the watch processor thread.
+                recoveryStatus.set(RecoveryStatus.IN_PROGRESS);
+
+                startWatchExecutor();
+
+                publishWatchEvent();
+
+                break;
+
+            default:
+                publishWatchEvent();
+
+                break;
+        }
+    }
+
+    private void publishWatchEvent() {
+        List<Entry> updatedEntriesCopy = List.copyOf(updatedEntries);
+
+        updatedEntries.clear();
+
+        try {
+            eventQueue.put(updatedEntriesCopy);
+        } catch (InterruptedException e) {
+            throw new MetaStorageException(WATCH_EXECUTION_ERR, "Interrupted when publishing a watch event", e);
+        }
     }
 
     /**
-     * Returns database.
+     * Notifies registered watches about an update event.
      */
-    RocksDB db() {
-        return db;
+    private void notifyWatches(List<Entry> updatedEntries) {
+        // Revision must be the same for all entries.
+        long newRevision = updatedEntries.get(0).revision();
+
+        for (Watch watch : watches) {
+            var entryEvents = new ArrayList<EntryEvent>();
+
+            for (Entry newEntry : updatedEntries) {
+                byte[] newKey = newEntry.key();
+
+                assert newEntry.revision() == newRevision;
+
+                if (watch.matches(newKey, newRevision)) {
+                    Entry oldEntry = get(newKey, newRevision - 1);
+
+                    entryEvents.add(new EntryEvent(oldEntry, newEntry));
+                }
+            }
+
+            if (!entryEvents.isEmpty()) {
+                var event = new WatchEvent(entryEvents, newRevision);
+
+                try {
+                    watch.onUpdate(event);
+                } catch (Exception e) {
+                    watch.onError(e);
+
+                    LOG.error("Error occurred when processing a watch event {}, watch will be disabled", e, event);
+
+                    watches.remove(watch);
+                }
+            }
+        }
+
+        // Watch processing for the new revision is done, notify the revision callback.
+        revisionCallback.accept(newRevision);
+    }
+
+    private void replayUpdates(long upperRevision) {
+        long minWatchRevision = watches.stream()
+                .mapToLong(Watch::targetRevision)
+                .min()
+                .orElse(-1);
+
+        if (minWatchRevision == -1 || minWatchRevision > upperRevision) {
+            return;
+        }
+
+        var updatedEntries = new ArrayList<Entry>();
+
+        try (
+                var upperBound = new Slice(longToBytes(upperRevision + 1));
+                var options = new ReadOptions().setIterateUpperBound(upperBound);
+                RocksIterator it = data.newIterator(options)
+        ) {
+            it.seek(longToBytes(minWatchRevision));
+
+            long lastSeenRevision = minWatchRevision;
+
+            for (; it.isValid(); it.next()) {
+                byte[] rocksKey = it.key();
+                byte[] rocksValue = it.value();
+
+                long revision = revisionFromRocksKey(rocksKey);
+
+                if (revision != lastSeenRevision) {
+                    if (!updatedEntries.isEmpty()) {
+                        var updatedEntriesCopy = List.copyOf(updatedEntries);
+
+                        watchExecutor.execute(() -> notifyWatches(updatedEntriesCopy));
+
+                        updatedEntries.clear();
+                    }
+
+                    lastSeenRevision = revision;
+                }
+
+                updatedEntries.add(entry(rocksKeyToBytes(rocksKey), revision, bytesToValue(rocksValue)));
+            }
+
+            RocksUtils.checkIterator(it);
+
+            // Notify about the events left after finishing the cycle above.
+            if (!updatedEntries.isEmpty()) {
+                watchExecutor.execute(() -> notifyWatches(updatedEntries));
+            }
+        }
+
+        // Events replay is finished, now we can start processing more recent events from the event queue.
+        startWatchExecutor();
+    }
+
+    private void startWatchExecutor() {
+        if (recoveryStatus.compareAndSet(RecoveryStatus.IN_PROGRESS, RecoveryStatus.DONE)) {
+            watchExecutorFuture = watchExecutor.submit(() -> {
+                while (!Thread.currentThread().isInterrupted()) {
+                    try {
+                        notifyWatches(eventQueue.take());
+                    } catch (InterruptedException e) {
+                        LOG.info("Watch Executor interrupted, watches stopped");
+
+                        return;
+                    }
+                }
+            });
+        }
+    }
+
+    /**
+     * Returns database lock.
+     */
+    ReadWriteLock lock() {
+        return rwLock;
     }
 
     @TestOnly
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
deleted file mode 100644
index c63f628715..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
+++ /dev/null
@@ -1,185 +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.internal.metastorage.server.persistence;
-
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.bytesToValue;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.longToBytes;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.revisionFromRocksKey;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.rocksKeyToBytes;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_EXECUTION_ERR;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.function.Predicate;
-import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
-import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.server.Value;
-import org.apache.ignite.internal.rocksdb.RocksUtils;
-import org.apache.ignite.internal.util.Cursor;
-import org.rocksdb.ReadOptions;
-import org.rocksdb.RocksDBException;
-import org.rocksdb.RocksIterator;
-
-/**
- * Subscription on updates of entries corresponding to the given keys range (where the upper bound is unlimited) and starting from the given
- * revision number.
- */
-class WatchCursor implements Cursor<WatchEvent> {
-    /** Storage. */
-    private final RocksDbKeyValueStorage storage;
-
-    /** Key predicate. */
-    private final Predicate<byte[]> predicate;
-
-    /** Options for {@link #nativeIterator}. */
-    private final ReadOptions options = new ReadOptions();
-
-    /** RocksDB iterator. */
-    private final RocksIterator nativeIterator;
-
-    /** Current revision. */
-    private long currentRevision;
-
-    /** Current value of the inner iterator's hasNext that is being reset to {@code false} after next is called. */
-    private boolean currentHasNext = false;
-
-    /**
-     * Constructor.
-     *
-     * @param storage Storage.
-     * @param rev     Starting revision.
-     * @param predicate       Key predicate.
-     */
-    WatchCursor(RocksDbKeyValueStorage storage, long rev, Predicate<byte[]> predicate) {
-        this.storage = storage;
-        this.predicate = predicate;
-
-        this.currentRevision = rev;
-
-        this.nativeIterator = storage.newDataIterator(options);
-        this.nativeIterator.seek(longToBytes(rev));
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public boolean hasNext() {
-        storage.lock().readLock().lock();
-
-        try {
-            if (currentHasNext) {
-                return true;
-            }
-
-            if (!nativeIterator.isValid()) {
-                nativeIterator.refresh();
-
-                nativeIterator.seek(longToBytes(currentRevision));
-            }
-
-            // Check all keys to see if any one of them match the predicate.
-            currentHasNext = RocksUtils.find(nativeIterator, (rocksKey, value) -> {
-                byte[] key = rocksKeyToBytes(rocksKey);
-
-                if (predicate.test(key)) {
-                    // We may have jumped to the next revision if there were no matching keys in previous.
-                    currentRevision = revisionFromRocksKey(rocksKey);
-
-                    return true;
-                }
-
-                return false;
-            });
-
-            return currentHasNext;
-        } catch (RocksDBException e) {
-            throw new MetaStorageException(WATCH_EXECUTION_ERR, e);
-        } finally {
-            storage.lock().readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public WatchEvent next() {
-        storage.lock().readLock().lock();
-
-        try {
-            if (!hasNext()) {
-                throw new NoSuchElementException();
-            }
-
-            List<EntryEvent> evts = new ArrayList<>();
-
-            long lastSeenRevision = currentRevision;
-
-            // Iterate over the keys of the current revision and get all matching entries.
-            for (; nativeIterator.isValid(); nativeIterator.next()) {
-                byte[] rocksKey = nativeIterator.key();
-                byte[] rocksValue = nativeIterator.value();
-
-                long revision = revisionFromRocksKey(rocksKey);
-
-                lastSeenRevision = revision;
-
-                if (revision > currentRevision) {
-                    // There are no more keys for the current revision
-                    break;
-                }
-
-                byte[] key = rocksKeyToBytes(rocksKey);
-
-                if (predicate.test(key)) {
-                    Value val = bytesToValue(rocksValue);
-
-                    Entry newEntry;
-
-                    if (val.tombstone()) {
-                        newEntry = EntryImpl.tombstone(key, revision, val.updateCounter());
-                    } else {
-                        newEntry = new EntryImpl(key, val.bytes(), revision, val.updateCounter());
-                    }
-
-                    Entry oldEntry = storage.doGet(key, revision - 1);
-
-                    evts.add(new EntryEvent(oldEntry, newEntry));
-                }
-            }
-
-            currentHasNext = false;
-
-            // Go to the next revision
-            currentRevision = lastSeenRevision > currentRevision ? lastSeenRevision : currentRevision + 1;
-
-            RocksUtils.checkIterator(nativeIterator);
-
-            return new WatchEvent(evts);
-        } finally {
-            storage.lock().readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void close() {
-        RocksUtils.closeAll(options, nativeIterator);
-    }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java
new file mode 100644
index 0000000000..7902e66781
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java
@@ -0,0 +1,73 @@
+/*
+ * 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.metastorage.server.raft;
+
+import java.nio.file.Path;
+import java.util.Iterator;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.raft.ReadCommand;
+import org.apache.ignite.internal.raft.WriteCommand;
+import org.apache.ignite.internal.raft.service.CommandClosure;
+import org.apache.ignite.internal.raft.service.RaftGroupListener;
+
+/**
+ * Meta Storage Raft group listener for learner nodes. These nodes ignore read and cursor-related commands.
+ */
+public class MetaStorageLearnerListener implements RaftGroupListener {
+    private final KeyValueStorage storage;
+
+    private final MetaStorageWriteHandler writeHandler;
+
+    public MetaStorageLearnerListener(KeyValueStorage storage) {
+        this.storage = storage;
+        this.writeHandler = new MetaStorageWriteHandler(storage);
+    }
+
+    @Override
+    public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
+        throw new UnsupportedOperationException("Reads should not be sent to learners");
+    }
+
+    @Override
+    public void onWrite(Iterator<CommandClosure<WriteCommand>> iter) {
+        while (iter.hasNext()) {
+            CommandClosure<WriteCommand> clo = iter.next();
+
+            if (!writeHandler.handleWriteCommand(clo)) {
+                // Ignore all commands that are not handled by the writeHandler.
+                clo.result(null);
+            }
+        }
+    }
+
+    @Override
+    public void onSnapshotSave(Path path, Consumer<Throwable> doneClo) {
+        storage.snapshot(path).whenComplete((unused, throwable) -> doneClo.accept(throwable));
+    }
+
+    @Override
+    public boolean onSnapshotLoad(Path path) {
+        storage.restoreSnapshot(path);
+        return true;
+    }
+
+    @Override
+    public void onShutdown() {
+    }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
index b894fde792..ded27cc751 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.metastorage.server.raft;
 
 import static java.util.Objects.requireNonNull;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CLOSING_STORAGE_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
 
 import java.nio.file.Path;
@@ -31,54 +30,18 @@ import java.util.NoSuchElementException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
 import org.apache.ignite.internal.metastorage.command.GetAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndRemoveCommand;
 import org.apache.ignite.internal.metastorage.command.GetCommand;
-import org.apache.ignite.internal.metastorage.command.InvokeCommand;
-import org.apache.ignite.internal.metastorage.command.MetaStorageCommandsFactory;
-import org.apache.ignite.internal.metastorage.command.MultiInvokeCommand;
 import org.apache.ignite.internal.metastorage.command.MultipleEntryResponse;
 import org.apache.ignite.internal.metastorage.command.PrefixCommand;
-import org.apache.ignite.internal.metastorage.command.PutAllCommand;
-import org.apache.ignite.internal.metastorage.command.PutCommand;
 import org.apache.ignite.internal.metastorage.command.RangeCommand;
-import org.apache.ignite.internal.metastorage.command.RemoveAllCommand;
-import org.apache.ignite.internal.metastorage.command.RemoveCommand;
 import org.apache.ignite.internal.metastorage.command.SingleEntryResponse;
-import org.apache.ignite.internal.metastorage.command.WatchExactKeysCommand;
-import org.apache.ignite.internal.metastorage.command.WatchRangeKeysCommand;
 import org.apache.ignite.internal.metastorage.command.cursor.CursorCloseCommand;
 import org.apache.ignite.internal.metastorage.command.cursor.CursorHasNextCommand;
 import org.apache.ignite.internal.metastorage.command.cursor.CursorNextCommand;
 import org.apache.ignite.internal.metastorage.command.cursor.CursorsCloseCommand;
-import org.apache.ignite.internal.metastorage.command.info.CompoundConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.ConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.IfInfo;
-import org.apache.ignite.internal.metastorage.command.info.OperationInfo;
-import org.apache.ignite.internal.metastorage.command.info.SimpleConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.StatementInfo;
-import org.apache.ignite.internal.metastorage.command.info.UpdateInfo;
-import org.apache.ignite.internal.metastorage.dsl.CompoundConditionType;
-import org.apache.ignite.internal.metastorage.dsl.ConditionType;
-import org.apache.ignite.internal.metastorage.dsl.Operation;
-import org.apache.ignite.internal.metastorage.dsl.StatementResult;
-import org.apache.ignite.internal.metastorage.dsl.Update;
 import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
-import org.apache.ignite.internal.metastorage.server.AndCondition;
-import org.apache.ignite.internal.metastorage.server.Condition;
-import org.apache.ignite.internal.metastorage.server.ExistenceCondition;
-import org.apache.ignite.internal.metastorage.server.If;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
-import org.apache.ignite.internal.metastorage.server.OrCondition;
-import org.apache.ignite.internal.metastorage.server.RevisionCondition;
-import org.apache.ignite.internal.metastorage.server.Statement;
-import org.apache.ignite.internal.metastorage.server.TombstoneCondition;
-import org.apache.ignite.internal.metastorage.server.ValueCondition;
 import org.apache.ignite.internal.raft.ReadCommand;
 import org.apache.ignite.internal.raft.WriteCommand;
 import org.apache.ignite.internal.raft.service.CommandClosure;
@@ -86,14 +49,13 @@ import org.apache.ignite.internal.raft.service.RaftGroupListener;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
-import org.jetbrains.annotations.TestOnly;
 
 /**
  * Meta storage listener.
  * TODO: IGNITE-14693 Implement Meta storage exception handling logic.
  */
 public class MetaStorageListener implements RaftGroupListener {
-    private final MetaStorageCommandsFactory commandsFactory = new MetaStorageCommandsFactory();
+    private final MetaStorageWriteHandler writeHandler;
 
     /** Storage. */
     private final KeyValueStorage storage;
@@ -108,6 +70,7 @@ public class MetaStorageListener implements RaftGroupListener {
      */
     public MetaStorageListener(KeyValueStorage storage) {
         this.storage = storage;
+        this.writeHandler = new MetaStorageWriteHandler(storage);
         this.cursors = new ConcurrentHashMap<>();
     }
 
@@ -130,9 +93,7 @@ public class MetaStorageListener implements RaftGroupListener {
                     e = storage.get(getCmd.key());
                 }
 
-                SingleEntryResponse resp = new SingleEntryResponse(
-                        e.key(), e.value(), e.revision(), e.updateCounter()
-                );
+                SingleEntryResponse resp = new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter());
 
                 clo.result(resp);
             } else if (command instanceof GetAllCommand) {
@@ -171,85 +132,13 @@ public class MetaStorageListener implements RaftGroupListener {
         while (iter.hasNext()) {
             CommandClosure<WriteCommand> clo = iter.next();
 
-            WriteCommand command = clo.command();
-
-            if (command instanceof PutCommand) {
-                PutCommand putCmd = (PutCommand) command;
-
-                storage.put(putCmd.key(), putCmd.value());
-
-                clo.result(null);
-            } else if (command instanceof GetAndPutCommand) {
-                GetAndPutCommand getAndPutCmd = (GetAndPutCommand) command;
-
-                Entry e = storage.getAndPut(getAndPutCmd.key(), getAndPutCmd.value());
-
-                clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
-            } else if (command instanceof PutAllCommand) {
-                PutAllCommand putAllCmd = (PutAllCommand) command;
-
-                storage.putAll(putAllCmd.keys(), putAllCmd.values());
-
-                clo.result(null);
-            } else if (command instanceof GetAndPutAllCommand) {
-                GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand) command;
-
-                Collection<Entry> entries = storage.getAndPutAll(getAndPutAllCmd.keys(), getAndPutAllCmd.values());
-
-                List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
-
-                for (Entry e : entries) {
-                    resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
-                }
-
-                clo.result(new MultipleEntryResponse(resp));
-            } else if (command instanceof RemoveCommand) {
-                RemoveCommand rmvCmd = (RemoveCommand) command;
-
-                storage.remove(rmvCmd.key());
-
-                clo.result(null);
-            } else if (command instanceof GetAndRemoveCommand) {
-                GetAndRemoveCommand getAndRmvCmd = (GetAndRemoveCommand) command;
-
-                Entry e = storage.getAndRemove(getAndRmvCmd.key());
-
-                clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
-            } else if (command instanceof RemoveAllCommand) {
-                RemoveAllCommand rmvAllCmd = (RemoveAllCommand) command;
-
-                storage.removeAll(rmvAllCmd.keys());
-
-                clo.result(null);
-            } else if (command instanceof GetAndRemoveAllCommand) {
-                GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand) command;
-
-                Collection<Entry> entries = storage.getAndRemoveAll(getAndRmvAllCmd.keys());
-
-                List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
-
-                for (Entry e : entries) {
-                    resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
-                }
-
-                clo.result(new MultipleEntryResponse(resp));
-            } else if (command instanceof InvokeCommand) {
-                InvokeCommand cmd = (InvokeCommand) command;
-
-                boolean res = storage.invoke(
-                        toCondition(cmd.condition()),
-                        toOperations(cmd.success()),
-                        toOperations(cmd.failure())
-                );
-
-                clo.result(res);
-            } else if (command instanceof MultiInvokeCommand) {
-                MultiInvokeCommand cmd = (MultiInvokeCommand) command;
+            if (writeHandler.handleWriteCommand(clo)) {
+                continue;
+            }
 
-                StatementResult res = storage.invoke(toIf(cmd.iif()));
+            WriteCommand command = clo.command();
 
-                clo.result(commandsFactory.statementResultInfo().result(res.bytes()).build());
-            } else if (command instanceof RangeCommand) {
+            if (command instanceof RangeCommand) {
                 RangeCommand rangeCmd = (RangeCommand) command;
 
                 IgniteUuid cursorId = rangeCmd.cursorId();
@@ -258,7 +147,7 @@ public class MetaStorageListener implements RaftGroupListener {
                         ? storage.range(rangeCmd.keyFrom(), rangeCmd.keyTo(), rangeCmd.revUpperBound(), rangeCmd.includeTombstones())
                         : storage.range(rangeCmd.keyFrom(), rangeCmd.keyTo(), rangeCmd.includeTombstones());
 
-                var cursorMeta = new CursorMeta(cursor, CursorType.RANGE, rangeCmd.requesterNodeId(), rangeCmd.batchSize());
+                var cursorMeta = new CursorMeta(cursor, rangeCmd.requesterNodeId(), rangeCmd.batchSize());
 
                 cursors.put(cursorId, cursorMeta);
 
@@ -272,7 +161,7 @@ public class MetaStorageListener implements RaftGroupListener {
                         ? storage.prefix(prefixCmd.prefix(), prefixCmd.includeTombstones())
                         : storage.prefix(prefixCmd.prefix(), prefixCmd.revUpperBound(), prefixCmd.includeTombstones());
 
-                var cursorMeta = new CursorMeta(cursor, CursorType.RANGE, prefixCmd.requesterNodeId(), prefixCmd.batchSize());
+                var cursorMeta = new CursorMeta(cursor, prefixCmd.requesterNodeId(), prefixCmd.batchSize());
 
                 cursors.put(cursorId, cursorMeta);
 
@@ -289,39 +178,19 @@ public class MetaStorageListener implements RaftGroupListener {
                 }
 
                 try {
-                    if (cursorDesc.type() == CursorType.RANGE) {
-                        int batchSize = requireNonNull(cursorDesc.batchSize());
-
-                        List<SingleEntryResponse> resp = new ArrayList<>(batchSize);
-
-                        for (int i = 0; i < batchSize; i++) {
-                            if (cursorDesc.cursor().hasNext()) {
-                                Entry e = (Entry) cursorDesc.cursor().next();
-
-                                resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
-                            } else {
-                                break;
-                            }
-                        }
-
-                        clo.result(new MultipleEntryResponse(resp));
-                    } else if (cursorDesc.type() == CursorType.WATCH) {
-                        WatchEvent evt = (WatchEvent) cursorDesc.cursor().next();
+                    int batchSize = requireNonNull(cursorDesc.batchSize());
 
-                        List<SingleEntryResponse> resp = new ArrayList<>(evt.entryEvents().size() * 2);
+                    var resp = new ArrayList<SingleEntryResponse>(batchSize);
 
-                        for (EntryEvent e : evt.entryEvents()) {
-                            Entry o = e.oldEntry();
+                    Cursor<Entry> cursor = cursorDesc.cursor();
 
-                            Entry n = e.newEntry();
+                    for (int i = 0; i < batchSize && cursor.hasNext(); i++) {
+                        Entry e = cursor.next();
 
-                            resp.add(new SingleEntryResponse(o.key(), o.value(), o.revision(), o.updateCounter()));
-
-                            resp.add(new SingleEntryResponse(n.key(), n.value(), n.revision(), n.updateCounter()));
-                        }
-
-                        clo.result(new MultipleEntryResponse(resp));
+                        resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
                     }
+
+                    clo.result(new MultipleEntryResponse(resp));
                 } catch (NoSuchElementException e) {
                     clo.result(e);
                 }
@@ -343,43 +212,6 @@ public class MetaStorageListener implements RaftGroupListener {
                 }
 
                 clo.result(null);
-            } else if (command instanceof WatchRangeKeysCommand) {
-                WatchRangeKeysCommand watchCmd = (WatchRangeKeysCommand) command;
-
-                IgniteUuid cursorId = watchCmd.cursorId();
-
-                Cursor<WatchEvent> cursor =
-                        storage.watch(watchCmd.keyFrom(), watchCmd.keyTo(), watchCmd.revision());
-
-                cursors.put(
-                        cursorId,
-                        new CursorMeta(
-                                cursor,
-                                CursorType.WATCH,
-                                watchCmd.requesterNodeId(),
-                                null
-                        )
-                );
-
-                clo.result(cursorId);
-            } else if (command instanceof WatchExactKeysCommand) {
-                WatchExactKeysCommand watchCmd = (WatchExactKeysCommand) command;
-
-                IgniteUuid cursorId = watchCmd.cursorId();
-
-                Cursor<WatchEvent> cursor = storage.watch(watchCmd.keys(), watchCmd.revision());
-
-                cursors.put(
-                        cursorId,
-                        new CursorMeta(
-                                cursor,
-                                CursorType.WATCH,
-                                watchCmd.requesterNodeId(),
-                                null
-                        )
-                );
-
-                clo.result(cursorId);
             } else if (command instanceof CursorsCloseCommand) {
                 CursorsCloseCommand cursorsCloseCmd = (CursorsCloseCommand) command;
 
@@ -425,112 +257,14 @@ public class MetaStorageListener implements RaftGroupListener {
     /** {@inheritDoc} */
     @Override
     public void onShutdown() {
-        try {
-            storage.close();
-        } catch (RuntimeException e) {
-            throw new MetaStorageException(CLOSING_STORAGE_ERR, "Failed to close storage: " + e.getMessage(), e);
-        }
-    }
-
-    /**
-     * Returns {@link KeyValueStorage} that is backing this listener.
-     */
-    @TestOnly
-    public KeyValueStorage getStorage() {
-        return storage;
-    }
-
-    private static If toIf(IfInfo iif) {
-        return new If(toCondition(iif.cond()), toConditionBranch(iif.andThen()), toConditionBranch(iif.orElse()));
-    }
-
-    private static Update toUpdate(UpdateInfo updateInfo) {
-        return new Update(toOperations(new ArrayList<>(updateInfo.operations())), new StatementResult(updateInfo.result().result()));
-    }
-
-    private static Statement toConditionBranch(StatementInfo statementInfo) {
-        if (statementInfo.isTerminal()) {
-            return new Statement(toUpdate(statementInfo.update()));
-        } else {
-            return new Statement(toIf(statementInfo.iif()));
-        }
-    }
-
-    private static Condition toCondition(ConditionInfo info) {
-        if (info instanceof SimpleConditionInfo) {
-            SimpleConditionInfo inf = (SimpleConditionInfo) info;
-            byte[] key = inf.key();
-
-            ConditionType type = inf.type();
-
-            if (type == ConditionType.KEY_EXISTS) {
-                return new ExistenceCondition(ExistenceCondition.Type.EXISTS, key);
-            } else if (type == ConditionType.KEY_NOT_EXISTS) {
-                return new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key);
-            } else if (type == ConditionType.TOMBSTONE) {
-                return new TombstoneCondition(key);
-            } else if (type == ConditionType.VAL_EQUAL) {
-                return new ValueCondition(ValueCondition.Type.EQUAL, key, inf.value());
-            } else if (type == ConditionType.VAL_NOT_EQUAL) {
-                return new ValueCondition(ValueCondition.Type.NOT_EQUAL, key, inf.value());
-            } else if (type == ConditionType.VAL_GREATER) {
-                return new ValueCondition(ValueCondition.Type.GREATER, key, inf.value());
-            } else if (type == ConditionType.VAL_GREATER_OR_EQUAL) {
-                return new ValueCondition(ValueCondition.Type.GREATER_OR_EQUAL, key, inf.value());
-            } else if (type == ConditionType.VAL_LESS) {
-                return new ValueCondition(ValueCondition.Type.LESS, key, inf.value());
-            } else if (type == ConditionType.VAL_LESS_OR_EQUAL) {
-                return new ValueCondition(ValueCondition.Type.LESS_OR_EQUAL, key, inf.value());
-            } else if (type == ConditionType.REV_EQUAL) {
-                return new RevisionCondition(RevisionCondition.Type.EQUAL, key, inf.revision());
-            } else if (type == ConditionType.REV_NOT_EQUAL) {
-                return new RevisionCondition(RevisionCondition.Type.NOT_EQUAL, key, inf.revision());
-            } else if (type == ConditionType.REV_GREATER) {
-                return new RevisionCondition(RevisionCondition.Type.GREATER, key, inf.revision());
-            } else if (type == ConditionType.REV_GREATER_OR_EQUAL) {
-                return new RevisionCondition(RevisionCondition.Type.GREATER_OR_EQUAL, key, inf.revision());
-            } else if (type == ConditionType.REV_LESS) {
-                return new RevisionCondition(RevisionCondition.Type.LESS, key, inf.revision());
-            } else if (type == ConditionType.REV_LESS_OR_EQUAL) {
-                return new RevisionCondition(RevisionCondition.Type.LESS_OR_EQUAL, key, inf.revision());
-            } else {
-                throw new IllegalArgumentException("Unknown condition type: " + type);
-            }
-        } else if (info instanceof CompoundConditionInfo) {
-            CompoundConditionInfo inf = (CompoundConditionInfo) info;
-
-            if (inf.type() == CompoundConditionType.AND) {
-                return new AndCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
-
-            } else if (inf.type() == CompoundConditionType.OR) {
-                return new OrCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
-            } else {
-                throw new IllegalArgumentException("Unknown compound condition " + inf.type());
-            }
-        } else {
-            throw new IllegalArgumentException("Unknown condition info type " + info);
-        }
-    }
-
-    private static List<Operation> toOperations(List<OperationInfo> infos) {
-        List<Operation> ops = new ArrayList<>(infos.size());
-
-        for (OperationInfo info : infos) {
-            ops.add(new Operation(info.type(), info.key(), info.value()));
-        }
-
-        return ops;
     }
 
     /**
      * Cursor meta information: origin node id and type.
      */
-    private class CursorMeta {
+    private static class CursorMeta {
         /** Cursor. */
-        private final Cursor<?> cursor;
-
-        /** Cursor type. */
-        private final CursorType type;
+        private final Cursor<Entry> cursor;
 
         /** Id of the node that creates cursor. */
         private final String requesterNodeId;
@@ -542,17 +276,11 @@ public class MetaStorageListener implements RaftGroupListener {
          * The constructor.
          *
          * @param cursor          Cursor.
-         * @param type            Cursor type.
          * @param requesterNodeId Id of the node that creates cursor.
          * @param batchSize       Batch size.
          */
-        CursorMeta(Cursor<?> cursor,
-                CursorType type,
-                String requesterNodeId,
-                @Nullable Integer batchSize
-        ) {
+        CursorMeta(Cursor<Entry> cursor, String requesterNodeId, @Nullable Integer batchSize) {
             this.cursor = cursor;
-            this.type = type;
             this.requesterNodeId = requesterNodeId;
             this.batchSize = batchSize;
         }
@@ -560,36 +288,22 @@ public class MetaStorageListener implements RaftGroupListener {
         /**
          * Returns cursor.
          */
-        public Cursor<?> cursor() {
+        Cursor<Entry> cursor() {
             return cursor;
         }
 
-        /**
-         * Returns cursor type.
-         */
-        public CursorType type() {
-            return type;
-        }
-
         /**
          * Returns id of the node that creates cursor.
          */
-        public String requesterNodeId() {
+        String requesterNodeId() {
             return requesterNodeId;
         }
 
         /**
          * Returns maximum size of the batch that is sent in single response message.
          */
-        public @Nullable Integer batchSize() {
+        @Nullable Integer batchSize() {
             return batchSize;
         }
     }
-
-    /** Cursor type. */
-    private enum CursorType {
-        RANGE,
-
-        WATCH;
-    }
 }
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java
new file mode 100644
index 0000000000..fbcebdbdc3
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java
@@ -0,0 +1,245 @@
+/*
+ * 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.metastorage.server.raft;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndRemoveCommand;
+import org.apache.ignite.internal.metastorage.command.InvokeCommand;
+import org.apache.ignite.internal.metastorage.command.MetaStorageCommandsFactory;
+import org.apache.ignite.internal.metastorage.command.MultiInvokeCommand;
+import org.apache.ignite.internal.metastorage.command.MultipleEntryResponse;
+import org.apache.ignite.internal.metastorage.command.PutAllCommand;
+import org.apache.ignite.internal.metastorage.command.PutCommand;
+import org.apache.ignite.internal.metastorage.command.RemoveAllCommand;
+import org.apache.ignite.internal.metastorage.command.RemoveCommand;
+import org.apache.ignite.internal.metastorage.command.SingleEntryResponse;
+import org.apache.ignite.internal.metastorage.command.info.CompoundConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.ConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.IfInfo;
+import org.apache.ignite.internal.metastorage.command.info.OperationInfo;
+import org.apache.ignite.internal.metastorage.command.info.SimpleConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.StatementInfo;
+import org.apache.ignite.internal.metastorage.command.info.UpdateInfo;
+import org.apache.ignite.internal.metastorage.dsl.CompoundConditionType;
+import org.apache.ignite.internal.metastorage.dsl.ConditionType;
+import org.apache.ignite.internal.metastorage.dsl.Operation;
+import org.apache.ignite.internal.metastorage.dsl.StatementResult;
+import org.apache.ignite.internal.metastorage.dsl.Update;
+import org.apache.ignite.internal.metastorage.server.AndCondition;
+import org.apache.ignite.internal.metastorage.server.Condition;
+import org.apache.ignite.internal.metastorage.server.ExistenceCondition;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.OrCondition;
+import org.apache.ignite.internal.metastorage.server.RevisionCondition;
+import org.apache.ignite.internal.metastorage.server.Statement;
+import org.apache.ignite.internal.metastorage.server.TombstoneCondition;
+import org.apache.ignite.internal.metastorage.server.ValueCondition;
+import org.apache.ignite.internal.raft.WriteCommand;
+import org.apache.ignite.internal.raft.service.CommandClosure;
+
+/**
+ * Class containing some common logic for Meta Storage Raft group listeners.
+ */
+class MetaStorageWriteHandler {
+    private final MetaStorageCommandsFactory commandsFactory = new MetaStorageCommandsFactory();
+
+    private final KeyValueStorage storage;
+
+    MetaStorageWriteHandler(KeyValueStorage storage) {
+        this.storage = storage;
+    }
+
+    /**
+     * Tries to process a {@link WriteCommand}, returning {@code true} if the command has been successfully processed or {@code false}
+     * if the command requires external processing.
+     */
+    boolean handleWriteCommand(CommandClosure<WriteCommand> clo) {
+        WriteCommand command = clo.command();
+
+        if (command instanceof PutCommand) {
+            PutCommand putCmd = (PutCommand) command;
+
+            storage.put(putCmd.key(), putCmd.value());
+
+            clo.result(null);
+        } else if (command instanceof GetAndPutCommand) {
+            GetAndPutCommand getAndPutCmd = (GetAndPutCommand) command;
+
+            Entry e = storage.getAndPut(getAndPutCmd.key(), getAndPutCmd.value());
+
+            clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+        } else if (command instanceof PutAllCommand) {
+            PutAllCommand putAllCmd = (PutAllCommand) command;
+
+            storage.putAll(putAllCmd.keys(), putAllCmd.values());
+
+            clo.result(null);
+        } else if (command instanceof GetAndPutAllCommand) {
+            GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand) command;
+
+            Collection<Entry> entries = storage.getAndPutAll(getAndPutAllCmd.keys(), getAndPutAllCmd.values());
+
+            List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
+
+            for (Entry e : entries) {
+                resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+            }
+
+            clo.result(new MultipleEntryResponse(resp));
+        } else if (command instanceof RemoveCommand) {
+            RemoveCommand rmvCmd = (RemoveCommand) command;
+
+            storage.remove(rmvCmd.key());
+
+            clo.result(null);
+        } else if (command instanceof GetAndRemoveCommand) {
+            GetAndRemoveCommand getAndRmvCmd = (GetAndRemoveCommand) command;
+
+            Entry e = storage.getAndRemove(getAndRmvCmd.key());
+
+            clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+        } else if (command instanceof RemoveAllCommand) {
+            RemoveAllCommand rmvAllCmd = (RemoveAllCommand) command;
+
+            storage.removeAll(rmvAllCmd.keys());
+
+            clo.result(null);
+        } else if (command instanceof GetAndRemoveAllCommand) {
+            GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand) command;
+
+            Collection<Entry> entries = storage.getAndRemoveAll(getAndRmvAllCmd.keys());
+
+            List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
+
+            for (Entry e : entries) {
+                resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+            }
+
+            clo.result(new MultipleEntryResponse(resp));
+        } else if (command instanceof InvokeCommand) {
+            InvokeCommand cmd = (InvokeCommand) command;
+
+            boolean res = storage.invoke(
+                    toCondition(cmd.condition()),
+                    toOperations(cmd.success()),
+                    toOperations(cmd.failure())
+            );
+
+            clo.result(res);
+        } else if (command instanceof MultiInvokeCommand) {
+            MultiInvokeCommand cmd = (MultiInvokeCommand) command;
+
+            StatementResult res = storage.invoke(toIf(cmd.iif()));
+
+            clo.result(commandsFactory.statementResultInfo().result(res.bytes()).build());
+        } else {
+            return false;
+        }
+
+        return true;
+    }
+
+    private static If toIf(IfInfo iif) {
+        return new If(toCondition(iif.cond()), toConditionBranch(iif.andThen()), toConditionBranch(iif.orElse()));
+    }
+
+    private static Update toUpdate(UpdateInfo updateInfo) {
+        return new Update(toOperations(new ArrayList<>(updateInfo.operations())), new StatementResult(updateInfo.result().result()));
+    }
+
+    private static Statement toConditionBranch(StatementInfo statementInfo) {
+        if (statementInfo.isTerminal()) {
+            return new Statement(toUpdate(statementInfo.update()));
+        } else {
+            return new Statement(toIf(statementInfo.iif()));
+        }
+    }
+
+    private static Condition toCondition(ConditionInfo info) {
+        if (info instanceof SimpleConditionInfo) {
+            SimpleConditionInfo inf = (SimpleConditionInfo) info;
+            byte[] key = inf.key();
+
+            ConditionType type = inf.type();
+
+            if (type == ConditionType.KEY_EXISTS) {
+                return new ExistenceCondition(ExistenceCondition.Type.EXISTS, key);
+            } else if (type == ConditionType.KEY_NOT_EXISTS) {
+                return new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key);
+            } else if (type == ConditionType.TOMBSTONE) {
+                return new TombstoneCondition(key);
+            } else if (type == ConditionType.VAL_EQUAL) {
+                return new ValueCondition(ValueCondition.Type.EQUAL, key, inf.value());
+            } else if (type == ConditionType.VAL_NOT_EQUAL) {
+                return new ValueCondition(ValueCondition.Type.NOT_EQUAL, key, inf.value());
+            } else if (type == ConditionType.VAL_GREATER) {
+                return new ValueCondition(ValueCondition.Type.GREATER, key, inf.value());
+            } else if (type == ConditionType.VAL_GREATER_OR_EQUAL) {
+                return new ValueCondition(ValueCondition.Type.GREATER_OR_EQUAL, key, inf.value());
+            } else if (type == ConditionType.VAL_LESS) {
+                return new ValueCondition(ValueCondition.Type.LESS, key, inf.value());
+            } else if (type == ConditionType.VAL_LESS_OR_EQUAL) {
+                return new ValueCondition(ValueCondition.Type.LESS_OR_EQUAL, key, inf.value());
+            } else if (type == ConditionType.REV_EQUAL) {
+                return new RevisionCondition(RevisionCondition.Type.EQUAL, key, inf.revision());
+            } else if (type == ConditionType.REV_NOT_EQUAL) {
+                return new RevisionCondition(RevisionCondition.Type.NOT_EQUAL, key, inf.revision());
+            } else if (type == ConditionType.REV_GREATER) {
+                return new RevisionCondition(RevisionCondition.Type.GREATER, key, inf.revision());
+            } else if (type == ConditionType.REV_GREATER_OR_EQUAL) {
+                return new RevisionCondition(RevisionCondition.Type.GREATER_OR_EQUAL, key, inf.revision());
+            } else if (type == ConditionType.REV_LESS) {
+                return new RevisionCondition(RevisionCondition.Type.LESS, key, inf.revision());
+            } else if (type == ConditionType.REV_LESS_OR_EQUAL) {
+                return new RevisionCondition(RevisionCondition.Type.LESS_OR_EQUAL, key, inf.revision());
+            } else {
+                throw new IllegalArgumentException("Unknown condition type: " + type);
+            }
+        } else if (info instanceof CompoundConditionInfo) {
+            CompoundConditionInfo inf = (CompoundConditionInfo) info;
+
+            if (inf.type() == CompoundConditionType.AND) {
+                return new AndCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
+
+            } else if (inf.type() == CompoundConditionType.OR) {
+                return new OrCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
+            } else {
+                throw new IllegalArgumentException("Unknown compound condition " + inf.type());
+            }
+        } else {
+            throw new IllegalArgumentException("Unknown condition info type " + info);
+        }
+    }
+
+    private static List<Operation> toOperations(List<OperationInfo> infos) {
+        List<Operation> ops = new ArrayList<>(infos.size());
+
+        for (OperationInfo info : infos) {
+            ops.add(new Operation(info.type(), info.key(), info.value()));
+        }
+
+        return ops;
+    }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
deleted file mode 100644
index d00ccb10f0..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
+++ /dev/null
@@ -1,68 +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.internal.metastorage.watch;
-
-import org.apache.ignite.internal.metastorage.WatchListener;
-
-/**
- * Watch implementation with associated revision. Instance of this watch produced by {@link WatchAggregator}.
- */
-public class AggregatedWatch {
-    /** Watch key criterion. */
-    private final KeyCriterion keyCriterion;
-
-    /** Aggregated watch listener. */
-    private final WatchListener lsnr;
-
-    /** Watch revision. */
-    private final long revision;
-
-    /**
-     * Creates the instance of aggregated watch.
-     *
-     * @param keyCriterion Aggregated key criterion.
-     * @param revision     Aggregated revision.
-     * @param lsnr         Aggregated listener.
-     */
-    public AggregatedWatch(KeyCriterion keyCriterion, long revision, WatchListener lsnr) {
-        this.keyCriterion = keyCriterion;
-        this.revision = revision;
-        this.lsnr = lsnr;
-    }
-
-    /**
-     * Returns key criterion.
-     */
-    public KeyCriterion keyCriterion() {
-        return keyCriterion;
-    }
-
-    /**
-     * Returns watch listener.
-     */
-    public WatchListener listener() {
-        return lsnr;
-    }
-
-    /**
-     * Returns revision.
-     */
-    public long revision() {
-        return revision;
-    }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
deleted file mode 100644
index df58d102f0..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
+++ /dev/null
@@ -1,369 +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.internal.metastorage.watch;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-import org.apache.ignite.lang.ByteArray;
-
-/**
- * Filter for listen key's changes on metastore.
- */
-public abstract class KeyCriterion {
-    /**
-     * Checks if this key criterion contains the key.
-     *
-     * @return {@code true} if criterion contains the key, {@code false} otherwise.
-     */
-    public abstract boolean contains(ByteArray key);
-
-    /**
-     * Union current key criterion with another one.
-     *
-     * @param keyCriterion Criterion to calculate the union with.
-     * @param swapTry      Set to {@code true} if current criterion can't calculate the union and trying to calculate it from the opposite
-     *                     side.
-     * @return Result key criterion.
-     */
-    protected abstract KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry);
-
-    /**
-     * Union two key criteria and produce the new one.
-     *
-     * <p>Rules for the union of different types of criteria:
-     * <pre>
-     * exact + exact = collection|exact
-     * collection + exact = collection
-     * collection + collection = collection
-     * range + exact = range
-     * range + collection = range
-     * range + range = range
-     * </pre>
-     *
-     * @param keyCriterion Criterion to calculate the union with.
-     * @return Result of criteria union.
-     */
-    public KeyCriterion union(KeyCriterion keyCriterion) {
-        return union(keyCriterion, false);
-    }
-
-    /**
-     * Creates a common exception that indicates the given key criterions cannot be combined.
-     *
-     * @param keyCriterion1 Criterion.
-     * @param keyCriterion2 Criterion.
-     * @return Common exception that indicates the given key criterions cannot be combined.
-     */
-    private static RuntimeException unsupportedUnionException(KeyCriterion keyCriterion1, KeyCriterion keyCriterion2) {
-        return new UnsupportedOperationException("Can't calculate the union between " + keyCriterion1.getClass()
-                + "and " + keyCriterion2.getClass() + " key criteria.");
-    }
-
-    /**
-     * Criterion which contains the range of keys.
-     */
-    public static class RangeCriterion extends KeyCriterion {
-        /** Start of the range. */
-        private final ByteArray from;
-
-        /** End of the range (exclusive). */
-        private final ByteArray to;
-
-        /**
-         * Creates the instance of range criterion.
-         *
-         * @param from Start of the range.
-         * @param to   End of the range (exclusive).
-         */
-        public RangeCriterion(ByteArray from, ByteArray to) {
-            this.from = from;
-            this.to = to;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean contains(ByteArray key) {
-            return key.compareTo(from) >= 0 && key.compareTo(to) < 0;
-        }
-
-        /**
-         * Calculates range representation for prefix criterion as {@code (prefixKey, nextKey(prefixKey)) }.
-         *
-         * @param prefixKey Prefix criterion.
-         * @return Calculated range
-         */
-        public static RangeCriterion fromPrefixKey(ByteArray prefixKey) {
-            return new RangeCriterion(prefixKey, nextKey(prefixKey));
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
-            ByteArray from;
-            ByteArray to;
-
-            if (keyCriterion instanceof ExactCriterion) {
-                from = ((ExactCriterion) keyCriterion).key;
-                to = nextKey(from);
-            } else if (keyCriterion instanceof CollectionCriterion) {
-                from = Collections.min(((CollectionCriterion) keyCriterion).keys);
-                to = nextKey(Collections.max(((CollectionCriterion) keyCriterion).keys));
-            } else if (keyCriterion instanceof RangeCriterion) {
-                from = ((RangeCriterion) keyCriterion).from;
-                to = ((RangeCriterion) keyCriterion).to;
-            } else if (!swapTry) {
-                return keyCriterion.union(this, true);
-            } else {
-                throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
-            }
-
-            return new RangeCriterion(
-                    minFromNullables(this.from, from),
-                    maxFromNullables(this.to, to)
-            );
-
-        }
-
-        /**
-         * Calculates the maximum of two keys in the scope of keys' range. According to the logic of range keys - null is an equivalent to
-         * +Inf in the range end position.
-         *
-         * @param key1 The first key to compare.
-         * @param key2 The second key to compare.
-         * @return Maximum key.
-         */
-        private static ByteArray maxFromNullables(ByteArray key1, ByteArray key2) {
-            if (key1 != null && key2 != null) {
-                return (key1.compareTo(key2) >= 0) ? key1 : key2;
-            } else {
-                return null;
-            }
-        }
-
-        /**
-         * Calculates the minimum of two keys in the scope of keys' range. According to the logic of range keys - null is an equivalent to
-         * -Inf in the range start position.
-         *
-         * @param key1 The first key to compare.
-         * @param key2 The second key to compare.
-         * @return Minimum key.
-         */
-        private static ByteArray minFromNullables(ByteArray key1, ByteArray key2) {
-            if (key1 != null && key2 != null) {
-                return (key1.compareTo(key2) < 0) ? key1 : key2;
-            } else {
-                return null;
-            }
-        }
-
-        /**
-         * Calculates the next key for received key.
-         *
-         * @param key Input key.
-         * @return Next key.
-         */
-        private static ByteArray nextKey(ByteArray key) {
-            var bytes = Arrays.copyOf(key.bytes(), key.bytes().length);
-
-            if (bytes[bytes.length - 1] != Byte.MAX_VALUE) {
-                bytes[bytes.length - 1]++;
-            } else {
-                bytes = Arrays.copyOf(bytes, bytes.length + 1);
-            }
-
-            return new ByteArray(bytes);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) {
-                return true;
-            }
-
-            if (o == null || getClass() != o.getClass()) {
-                return false;
-            }
-
-            RangeCriterion criterion = (RangeCriterion) o;
-            return Objects.equals(from, criterion.from) && Objects.equals(to, criterion.to);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public int hashCode() {
-            return Objects.hash(from, to);
-        }
-
-        /**
-         * Returns start of the range.
-         */
-        public ByteArray from() {
-            return from;
-        }
-
-        /**
-         * Returns end of the range (exclusive).
-         */
-        public ByteArray to() {
-            return to;
-        }
-    }
-
-    /**
-     * Criterion which consists collection of keys.
-     */
-    public static class CollectionCriterion extends KeyCriterion {
-        /** Collection of keys. */
-        private final Set<ByteArray> keys;
-
-        /**
-         * Creates the instance of collection criterion.
-         *
-         * @param keys Collection of keys.
-         */
-        public CollectionCriterion(Collection<ByteArray> keys) {
-            this.keys = new HashSet<>(keys);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean contains(ByteArray key) {
-            return keys.contains(key);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
-            var newKeys = new HashSet<>(keys);
-
-            if (keyCriterion instanceof ExactCriterion) {
-                newKeys.add(((ExactCriterion) keyCriterion).key);
-
-                return new CollectionCriterion(newKeys);
-            } else if (keyCriterion instanceof CollectionCriterion) {
-                newKeys.addAll(((CollectionCriterion) keyCriterion).keys);
-
-                return new CollectionCriterion(newKeys);
-            } else if (!swapTry) {
-                return keyCriterion.union(keyCriterion, true);
-            } else {
-                throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) {
-                return true;
-            }
-
-            if (o == null || getClass() != o.getClass()) {
-                return false;
-            }
-
-            CollectionCriterion criterion = (CollectionCriterion) o;
-            return Objects.equals(keys, criterion.keys);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public int hashCode() {
-            return Objects.hash(keys);
-        }
-
-        /**
-         * Returns collection of keys.
-         */
-        public Set<ByteArray> keys() {
-            return keys;
-        }
-    }
-
-    /**
-     * Simple criterion which contains exactly one key.
-     */
-    public static class ExactCriterion extends KeyCriterion {
-        /** The key of criterion. */
-        private final ByteArray key;
-
-        /**
-         * Creates the instance of exact criterion.
-         *
-         * @param key Instance of the reference key.
-         */
-        public ExactCriterion(ByteArray key) {
-            this.key = key;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean contains(ByteArray key) {
-            return this.key.equals(key);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
-            if (keyCriterion instanceof ExactCriterion) {
-                if (equals(keyCriterion)) {
-                    return this;
-                } else {
-                    return new CollectionCriterion(Arrays.asList(key, ((ExactCriterion) keyCriterion).key));
-                }
-            } else if (!swapTry) {
-                return keyCriterion.union(this, true);
-            } else {
-                throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) {
-                return true;
-            }
-
-            if (o == null || getClass() != o.getClass()) {
-                return false;
-            }
-
-            ExactCriterion criterion = (ExactCriterion) o;
-            return Objects.equals(key, criterion.key);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public int hashCode() {
-            return Objects.hash(key);
-        }
-
-        /**
-         * Returns the key of criterion.
-         */
-        public ByteArray key() {
-            return key;
-        }
-    }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
deleted file mode 100644
index 4cbd89c6c7..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
+++ /dev/null
@@ -1,263 +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.internal.metastorage.watch;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.BiConsumer;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * Needed to aggregate multiple watches to one aggregated watch. This approach needed to provide the following additional guarantees to
- * watching mechanism: - watch events will be processed sequentially - watch events will be resolved in the order of watch registration
- */
-public class WatchAggregator {
-    /**
-     * Watches' map must be synchronized because of changes from WatchListener in separate thread.
-     */
-    private final Map<Long, Watch> watches = Collections.synchronizedMap(new LinkedHashMap<>());
-
-    /** Simple auto increment id for internal watches. */
-    private final AtomicLong idCntr = new AtomicLong(0);
-
-    /**
-     * Adds new watch with simple exact criterion.
-     *
-     * @param key  Key for watching.
-     * @param lsnr Listener which will be executed on watch event.
-     * @return id of registered watch. Can be used for remove watch from later.
-     */
-    public long add(ByteArray key, WatchListener lsnr) {
-        var watch = new Watch(new KeyCriterion.ExactCriterion(key), lsnr);
-        var id = idCntr.incrementAndGet();
-        watches.put(id, watch);
-        return id;
-    }
-
-    /**
-     * Adds new watch with filter by collection of keys.
-     *
-     * @param keys Collection of keys to listen.
-     * @param lsnr Listener which will be executed on watch event.
-     * @return id of registered watch. Can be used for remove watch from later.
-     */
-    public long add(Collection<ByteArray> keys, WatchListener lsnr) {
-        var watch = new Watch(new KeyCriterion.CollectionCriterion(keys), lsnr);
-        var id = idCntr.incrementAndGet();
-        watches.put(id, watch);
-        return id;
-    }
-
-    /**
-     * Adds new watch with filter by collection of keys.
-     *
-     * @param from Start key of range to listen.
-     * @param to   End key of range (exclusively)..
-     * @param lsnr Listener which will be executed on watch event.
-     * @return id of registered watch. Can be used for remove watch from later.
-     */
-    public long add(ByteArray from, ByteArray to, WatchListener lsnr) {
-        var watch = new Watch(new KeyCriterion.RangeCriterion(from, to), lsnr);
-        var id = idCntr.incrementAndGet();
-        watches.put(id, watch);
-        return id;
-    }
-
-    /**
-     * Adds new watch with filter by key prefix.
-     *
-     * @param key  Prefix for key.
-     * @param lsnr Listener which will be executed on watch event.
-     * @return id of registered watch. Can be used for remove watch from later.
-     */
-    public long addPrefix(ByteArray key, WatchListener lsnr) {
-        var watch = new Watch(KeyCriterion.RangeCriterion.fromPrefixKey(key), lsnr);
-        var id = idCntr.incrementAndGet();
-        watches.put(id, watch);
-        return id;
-    }
-
-    /**
-     * Cancel watch by id.
-     *
-     * @param id of watch to cancel.
-     */
-    public void cancel(long id) {
-        watches.remove(id);
-    }
-
-    /**
-     * Cancel multiple watches by ids.
-     *
-     * @param ids of watches to cancel.
-     */
-    public void cancelAll(Collection<Long> ids) {
-        watches.keySet().removeAll(ids);
-    }
-
-    /**
-     * Produce watch with aggregated key criterion and general watch listener dispatcher.
-     *
-     * @param revision        start revision to listen event.
-     * @param saveRevisionAct action to commit keys-revision pair to persistent store for processed keys.
-     * @return result aggregated watch.
-     */
-    public Optional<AggregatedWatch> watch(
-            long revision,
-            BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> saveRevisionAct
-    ) {
-        synchronized (watches) {
-            if (watches.isEmpty()) {
-                return Optional.empty();
-            } else {
-                return Optional.of(new AggregatedWatch(inferGeneralCriteria(), revision, watchListener(saveRevisionAct)));
-            }
-        }
-    }
-
-    /**
-     * Returns general criterion, which overlays all aggregated criteria.
-     *
-     * @return aggregated criterion.
-     */
-    private KeyCriterion inferGeneralCriteria() {
-        return
-                watches.values().stream()
-                        .map(Watch::keyCriterion)
-                        .reduce(KeyCriterion::union)
-                        .get();
-    }
-
-    /**
-     * Produces the watch listener, which will dispatch events to appropriate watches.
-     *
-     * @param storeRevision action to commit keys-revision pair to persistent store for processed keys.
-     * @return watch listener, which will dispatch events to appropriate watches.
-     */
-    private WatchListener watchListener(BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> storeRevision) {
-        // Copy watches to separate collection, because all changes on the WatchAggregator watches
-        // shouldn't be propagated to listener watches immediately.
-        // WatchAggregator will be redeployed with new watches if needed instead.
-        final LinkedHashMap<Long, Watch> cpWatches = new LinkedHashMap<>(watches);
-
-        return new WatchListener() {
-
-            @Override
-            public boolean onUpdate(@NotNull WatchEvent evt) {
-                //TODO: IGNITE-15858 Fix stopWatch may solve the issue.
-                synchronized (watches) {
-                    processWatchEvents(evt);
-                }
-
-                return true;
-            }
-
-            /**
-             * Process watch events synchronously.
-             *
-             * @param evt Watch event.
-             */
-            private void processWatchEvents(@NotNull WatchEvent evt) {
-                var watchIt = cpWatches.entrySet().iterator();
-                Collection<Long> toCancel = new ArrayList<>();
-
-                while (watchIt.hasNext()) {
-                    Map.Entry<Long, Watch> entry = watchIt.next();
-                    Watch watch = entry.getValue();
-                    var filteredEvts = new ArrayList<EntryEvent>();
-
-                    for (EntryEvent entryEvt : evt.entryEvents()) {
-                        if (watch.keyCriterion().contains(new ByteArray(entryEvt.oldEntry().key()))) {
-                            filteredEvts.add(entryEvt);
-                        }
-                    }
-
-                    if (!filteredEvts.isEmpty()) {
-                        if (!watch.listener().onUpdate(new WatchEvent(filteredEvts))) {
-                            watchIt.remove();
-
-                            toCancel.add(entry.getKey());
-                        }
-                    }
-                }
-
-                // Cancel finished watches from the global watch map
-                // to prevent finished watches from redeploy.
-                if (!toCancel.isEmpty()) {
-                    cancelAll(toCancel);
-                }
-
-                var revision = 0L;
-                var entries = new ArrayList<IgniteBiTuple<ByteArray, byte[]>>();
-                for (EntryEvent entryEvt : evt.entryEvents()) {
-                    revision = entryEvt.newEntry().revision();
-
-                    entries.add(new IgniteBiTuple<>(new ByteArray(entryEvt.newEntry().key()), entryEvt.newEntry().value()));
-                }
-
-                storeRevision.accept(entries, revision);
-            }
-
-            @Override
-            public void onError(@NotNull Throwable e) {
-                watches.values().forEach(w -> w.listener().onError(e));
-            }
-        };
-    }
-
-    /**
-     * (key criterion, watch listener) container.
-     */
-    private static class Watch {
-        /** Key criterion. */
-        private final KeyCriterion keyCriterion;
-
-        /** Watch listener. */
-        private final WatchListener lsnr;
-
-        /** Creates the watch. */
-        private Watch(KeyCriterion keyCriterion, WatchListener lsnr) {
-            this.keyCriterion = keyCriterion;
-            this.lsnr = lsnr;
-        }
-
-        /**
-         * Returns key criterion.
-         */
-        public KeyCriterion keyCriterion() {
-            return keyCriterion;
-        }
-
-        /**
-         * Returns watch listener.
-         */
-        public WatchListener listener() {
-            return lsnr;
-        }
-    }
-}
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
index d948732013..462ffc6269 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
@@ -49,6 +49,8 @@ import org.apache.ignite.internal.raft.service.CommandClosure;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.extension.ExtendWith;
 import org.junit.jupiter.params.ParameterizedTest;
@@ -62,13 +64,13 @@ import org.mockito.junit.jupiter.MockitoExtension;
  */
 @ExtendWith(MockitoExtension.class)
 public class MetaStorageRangeCursorTest {
-    MetaStorageListener listener;
+    private MetaStorageListener listener;
 
     @Mock
-    RaftGroupService raftGroupService;
+    private RaftGroupService raftGroupService;
 
     @Mock
-    KeyValueStorage storage;
+    private KeyValueStorage storage;
 
     @ParameterizedTest
     @MethodSource("getParameters")
@@ -82,7 +84,9 @@ public class MetaStorageRangeCursorTest {
 
         when(raftGroupService.run(any())).thenAnswer(invocation -> runCommand(invocation.getArgument(0)));
 
-        MetaStorageService metaStorageService = new MetaStorageServiceImpl(raftGroupService, "test", "test");
+        var localNode = new ClusterNode("test", "test", new NetworkAddress("localhost", 10000));
+
+        MetaStorageService metaStorageService = new MetaStorageServiceImpl(raftGroupService, localNode);
 
         checkCursor(metaStorageService.range(intToBytes(0), intToBytes(keyTo)), limit);
         checkCursor(metaStorageService.range(intToBytes(0), intToBytes(keyTo), 0), limit);
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
index e4736c8a5c..b918e31da7 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
@@ -19,25 +19,30 @@ package org.apache.ignite.internal.metastorage.server;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.function.Function.identity;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.nio.ByteBuffer;
 import java.util.Collection;
-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.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.EntryEvent;
 import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.dsl.OperationType;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
@@ -45,6 +50,7 @@ import org.apache.ignite.internal.metastorage.dsl.Update;
 import org.apache.ignite.internal.metastorage.server.ValueCondition.Type;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -53,7 +59,7 @@ import org.junit.jupiter.api.Test;
  * Tests for key-value storage implementations.
  */
 public abstract class AbstractKeyValueStorageTest {
-    private KeyValueStorage storage;
+    protected KeyValueStorage storage;
 
     /**
      * Before each.
@@ -1949,88 +1955,84 @@ public abstract class AbstractKeyValueStorageTest {
         assertEquals(3, storage.updateCounter());
 
         // Range for latest revision without max bound.
-        Cursor<Entry> cur = storage.range(key1, null, false);
+        try (Cursor<Entry> cur = storage.range(key1, null, false)) {
+            assertTrue(cur.hasNext());
 
-        Iterator<Entry> it = cur.iterator();
+            Entry e1 = cur.next();
 
-        assertTrue(it.hasNext());
+            assertFalse(e1.empty());
+            assertFalse(e1.tombstone());
+            assertArrayEquals(key1, e1.key());
+            assertArrayEquals(val1, e1.value());
+            assertEquals(1, e1.revision());
+            assertEquals(1, e1.updateCounter());
 
-        Entry e1 = it.next();
+            assertTrue(cur.hasNext());
 
-        assertFalse(e1.empty());
-        assertFalse(e1.tombstone());
-        assertArrayEquals(key1, e1.key());
-        assertArrayEquals(val1, e1.value());
-        assertEquals(1, e1.revision());
-        assertEquals(1, e1.updateCounter());
+            Entry e2 = cur.next();
 
-        assertTrue(it.hasNext());
+            assertFalse(e2.empty());
+            assertFalse(e2.tombstone());
+            assertArrayEquals(key2, e2.key());
+            assertArrayEquals(val2, e2.value());
+            assertEquals(1, e2.revision());
+            assertEquals(2, e2.updateCounter());
 
-        Entry e2 = it.next();
+            // Deliberately don't call cur.hasNext()
 
-        assertFalse(e2.empty());
-        assertFalse(e2.tombstone());
-        assertArrayEquals(key2, e2.key());
-        assertArrayEquals(val2, e2.value());
-        assertEquals(1, e2.revision());
-        assertEquals(2, e2.updateCounter());
-
-        // Deliberately don't call it.hasNext()
-
-        Entry e3 = it.next();
+            Entry e3 = cur.next();
 
-        assertFalse(e3.empty());
-        assertFalse(e3.tombstone());
-        assertArrayEquals(key3, e3.key());
-        assertArrayEquals(val3, e3.value());
-        assertEquals(1, e3.revision());
-        assertEquals(3, e3.updateCounter());
+            assertFalse(e3.empty());
+            assertFalse(e3.tombstone());
+            assertArrayEquals(key3, e3.key());
+            assertArrayEquals(val3, e3.value());
+            assertEquals(1, e3.revision());
+            assertEquals(3, e3.updateCounter());
 
-        assertFalse(it.hasNext());
+            assertFalse(cur.hasNext());
 
-        try {
-            it.next();
+            try {
+                cur.next();
 
-            fail();
-        } catch (NoSuchElementException e) {
-            // No-op.
+                fail();
+            } catch (NoSuchElementException e) {
+                // No-op.
+            }
         }
 
         // Range for latest revision with max bound.
-        cur = storage.range(key1, key3, false);
-
-        it = cur.iterator();
+        try (Cursor<Entry> cur = storage.range(key1, key3, false)) {
+            assertTrue(cur.hasNext());
 
-        assertTrue(it.hasNext());
+            Entry e1 = cur.next();
 
-        e1 = it.next();
+            assertFalse(e1.empty());
+            assertFalse(e1.tombstone());
+            assertArrayEquals(key1, e1.key());
+            assertArrayEquals(val1, e1.value());
+            assertEquals(1, e1.revision());
+            assertEquals(1, e1.updateCounter());
 
-        assertFalse(e1.empty());
-        assertFalse(e1.tombstone());
-        assertArrayEquals(key1, e1.key());
-        assertArrayEquals(val1, e1.value());
-        assertEquals(1, e1.revision());
-        assertEquals(1, e1.updateCounter());
+            assertTrue(cur.hasNext());
 
-        assertTrue(it.hasNext());
+            Entry e2 = cur.next();
 
-        e2 = it.next();
+            assertFalse(e2.empty());
+            assertFalse(e2.tombstone());
+            assertArrayEquals(key2, e2.key());
+            assertArrayEquals(val2, e2.value());
+            assertEquals(1, e2.revision());
+            assertEquals(2, e2.updateCounter());
 
-        assertFalse(e2.empty());
-        assertFalse(e2.tombstone());
-        assertArrayEquals(key2, e2.key());
-        assertArrayEquals(val2, e2.value());
-        assertEquals(1, e2.revision());
-        assertEquals(2, e2.updateCounter());
-
-        assertFalse(it.hasNext());
+            assertFalse(cur.hasNext());
 
-        try {
-            it.next();
+            try {
+                cur.next();
 
-            fail();
-        } catch (NoSuchElementException e) {
-            // No-op.
+                fail();
+            } catch (NoSuchElementException e) {
+                // No-op.
+            }
         }
     }
 
@@ -2079,7 +2081,7 @@ public abstract class AbstractKeyValueStorageTest {
     }
 
     @Test
-    public void watchCursorLexicographicTest() throws Exception {
+    public void watchLexicographicTest() {
         assertEquals(0, storage.revision());
         assertEquals(0, storage.updateCounter());
 
@@ -2088,37 +2090,30 @@ public abstract class AbstractKeyValueStorageTest {
 
         int count = 1000; // Exceeds 1 byte
 
-        for (int i = 0; i < count; i++) {
-            storage.put(key, val);
-        }
-
-        assertEquals(count, storage.revision());
-        assertEquals(count, storage.updateCounter());
-
-        int i = 1;
-        int countSeen = 0;
-
-        try (Cursor<WatchEvent> cur = storage.watch(key, 1)) {
-            for (WatchEvent event : cur) {
-                assertTrue(event.single());
+        CompletableFuture<Void> awaitFuture = watchExact(key, 1, count, (event, state) -> {
+            assertTrue(event.single());
 
-                Entry entry = event.entryEvent().newEntry();
+            Entry entry = event.entryEvent().newEntry();
 
-                byte[] entryKey = entry.key();
+            byte[] entryKey = entry.key();
 
-                assertEquals(i++, entry.revision());
+            assertEquals((long) state, entry.revision());
 
-                assertArrayEquals(key, entryKey);
+            assertArrayEquals(key, entryKey);
+        });
 
-                countSeen++;
-            }
+        for (int i = 0; i < count; i++) {
+            storage.put(key, val);
         }
 
-        assertEquals(count, countSeen);
+        assertEquals(count, storage.revision());
+        assertEquals(count, storage.updateCounter());
+
+        assertThat(awaitFuture, willCompleteSuccessfully());
     }
 
     @Test
-    public void watchCursorForRange() throws Exception {
+    public void testWatchRange() {
         byte[] key1 = key(1);
         byte[] val11 = keyValue(1, 11);
 
@@ -2133,119 +2128,99 @@ public abstract class AbstractKeyValueStorageTest {
         assertEquals(0, storage.updateCounter());
 
         // Watch for all updates starting from revision 2.
-        Cursor<WatchEvent> cur = storage.watch(key1, null, 2);
+        CompletableFuture<Void> awaitFuture = watchRange(key1, null, 2, 2, (event, state) -> {
+            if (state == 1) {
+                assertFalse(event.single());
 
-        Iterator<WatchEvent> it = cur.iterator();
+                Map<ByteArray, EntryEvent> map = event.entryEvents().stream()
+                        .collect(Collectors.toMap(evt -> new ByteArray(evt.newEntry().key()), identity()));
 
-        assertFalse(it.hasNext());
-        assertThrows(NoSuchElementException.class, it::next);
+                assertEquals(2, map.size());
 
-        storage.putAll(List.of(key1, key2), List.of(val11, val21));
+                // First update under revision.
+                EntryEvent e2 = map.get(new ByteArray(key2));
 
-        assertEquals(1, storage.revision());
-        assertEquals(2, storage.updateCounter());
+                assertNotNull(e2);
 
-        // Revision is less than 2.
-        assertFalse(it.hasNext());
-        assertThrows(NoSuchElementException.class, it::next);
+                Entry oldEntry2 = e2.oldEntry();
 
-        storage.putAll(List.of(key2, key3), List.of(val22, val31));
+                assertFalse(oldEntry2.empty());
+                assertFalse(oldEntry2.tombstone());
+                assertEquals(1, oldEntry2.revision());
+                assertEquals(2, oldEntry2.updateCounter());
+                assertArrayEquals(key2, oldEntry2.key());
+                assertArrayEquals(val21, oldEntry2.value());
 
-        assertEquals(2, storage.revision());
-        assertEquals(4, storage.updateCounter());
+                Entry newEntry2 = e2.newEntry();
 
-        // Revision is 2.
-        assertTrue(it.hasNext());
+                assertFalse(newEntry2.empty());
+                assertFalse(newEntry2.tombstone());
+                assertEquals(2, newEntry2.revision());
+                assertEquals(3, newEntry2.updateCounter());
+                assertArrayEquals(key2, newEntry2.key());
+                assertArrayEquals(val22, newEntry2.value());
 
-        WatchEvent watchEvent = it.next();
+                // Second update under revision.
+                EntryEvent e3 = map.get(new ByteArray(key3));
 
-        assertFalse(watchEvent.single());
+                assertNotNull(e3);
 
-        Map<ByteArray, EntryEvent> map = watchEvent.entryEvents().stream()
-                .collect(Collectors.toMap(evt -> new ByteArray(evt.newEntry().key()), identity()));
+                Entry oldEntry3 = e3.oldEntry();
 
-        assertEquals(2, map.size());
+                assertTrue(oldEntry3.empty());
+                assertFalse(oldEntry3.tombstone());
+                assertArrayEquals(key3, oldEntry3.key());
 
-        // First update under revision.
-        EntryEvent e2 = map.get(new ByteArray(key2));
-
-        assertNotNull(e2);
+                Entry newEntry3 = e3.newEntry();
 
-        Entry oldEntry2 = e2.oldEntry();
-
-        assertFalse(oldEntry2.empty());
-        assertFalse(oldEntry2.tombstone());
-        assertEquals(1, oldEntry2.revision());
-        assertEquals(2, oldEntry2.updateCounter());
-        assertArrayEquals(key2, oldEntry2.key());
-        assertArrayEquals(val21, oldEntry2.value());
+                assertFalse(newEntry3.empty());
+                assertFalse(newEntry3.tombstone());
+                assertEquals(2, newEntry3.revision());
+                assertEquals(4, newEntry3.updateCounter());
+                assertArrayEquals(key3, newEntry3.key());
+                assertArrayEquals(val31, newEntry3.value());
+            } else if (state == 2) {
+                assertTrue(event.single());
 
-        Entry newEntry2 = e2.newEntry();
+                EntryEvent e1 = event.entryEvent();
 
-        assertFalse(newEntry2.empty());
-        assertFalse(newEntry2.tombstone());
-        assertEquals(2, newEntry2.revision());
-        assertEquals(3, newEntry2.updateCounter());
-        assertArrayEquals(key2, newEntry2.key());
-        assertArrayEquals(val22, newEntry2.value());
+                Entry oldEntry1 = e1.oldEntry();
 
-        // Second update under revision.
-        EntryEvent e3 = map.get(new ByteArray(key3));
+                assertFalse(oldEntry1.empty());
+                assertFalse(oldEntry1.tombstone());
+                assertEquals(1, oldEntry1.revision());
+                assertEquals(1, oldEntry1.updateCounter());
+                assertArrayEquals(key1, oldEntry1.key());
+                assertArrayEquals(val11, oldEntry1.value());
 
-        assertNotNull(e3);
+                Entry newEntry1 = e1.newEntry();
 
-        Entry oldEntry3 = e3.oldEntry();
+                assertFalse(newEntry1.empty());
+                assertTrue(newEntry1.tombstone());
+                assertEquals(3, newEntry1.revision());
+                assertEquals(5, newEntry1.updateCounter());
+                assertArrayEquals(key1, newEntry1.key());
+                assertNull(newEntry1.value());
+            }
+        });
 
-        assertTrue(oldEntry3.empty());
-        assertFalse(oldEntry3.tombstone());
-        assertArrayEquals(key3, oldEntry3.key());
+        storage.putAll(List.of(key1, key2), List.of(val11, val21));
 
-        Entry newEntry3 = e3.newEntry();
+        assertEquals(1, storage.revision());
+        assertEquals(2, storage.updateCounter());
 
-        assertFalse(newEntry3.empty());
-        assertFalse(newEntry3.tombstone());
-        assertEquals(2, newEntry3.revision());
-        assertEquals(4, newEntry3.updateCounter());
-        assertArrayEquals(key3, newEntry3.key());
-        assertArrayEquals(val31, newEntry3.value());
+        storage.putAll(List.of(key2, key3), List.of(val22, val31));
 
-        assertFalse(it.hasNext());
+        assertEquals(2, storage.revision());
+        assertEquals(4, storage.updateCounter());
 
         storage.remove(key1);
 
-        assertTrue(it.hasNext());
-
-        watchEvent = it.next();
-
-        assertTrue(watchEvent.single());
-
-        EntryEvent e1 = watchEvent.entryEvent();
-
-        Entry oldEntry1 = e1.oldEntry();
-
-        assertFalse(oldEntry1.empty());
-        assertFalse(oldEntry1.tombstone());
-        assertEquals(1, oldEntry1.revision());
-        assertEquals(1, oldEntry1.updateCounter());
-        assertArrayEquals(key1, oldEntry1.key());
-        assertArrayEquals(val11, oldEntry1.value());
-
-        Entry newEntry1 = e1.newEntry();
-
-        assertFalse(newEntry1.empty());
-        assertTrue(newEntry1.tombstone());
-        assertEquals(3, newEntry1.revision());
-        assertEquals(5, newEntry1.updateCounter());
-        assertArrayEquals(key1, newEntry1.key());
-        assertNull(newEntry1.value());
-
-        assertFalse(it.hasNext());
-
-        cur.close();
+        assertThat(awaitFuture, willCompleteSuccessfully());
     }
 
     @Test
-    public void watchCursorForKey() {
+    public void testWatchExact() {
         byte[] key1 = key(1);
         byte[] val11 = keyValue(1, 11);
         byte[] val12 = keyValue(1, 12);
@@ -2257,79 +2232,63 @@ public abstract class AbstractKeyValueStorageTest {
         assertEquals(0, storage.revision());
         assertEquals(0, storage.updateCounter());
 
-        Cursor<WatchEvent> cur = storage.watch(key1, 1);
+        CompletableFuture<Void> awaitFuture = watchExact(key1, 1, 2, (event, state) -> {
+            if (state == 1) {
+                assertTrue(event.single());
 
-        Iterator<WatchEvent> it = cur.iterator();
+                EntryEvent e1 = event.entryEvent();
 
-        assertFalse(it.hasNext());
-        assertThrows(NoSuchElementException.class, it::next);
+                Entry oldEntry1 = e1.oldEntry();
 
-        storage.putAll(List.of(key1, key2), List.of(val11, val21));
+                assertTrue(oldEntry1.empty());
+                assertFalse(oldEntry1.tombstone());
 
-        assertEquals(1, storage.revision());
-        assertEquals(2, storage.updateCounter());
+                Entry newEntry1 = e1.newEntry();
 
-        assertTrue(it.hasNext());
-
-        WatchEvent watchEvent = it.next();
+                assertFalse(newEntry1.empty());
+                assertFalse(newEntry1.tombstone());
+                assertEquals(1, newEntry1.revision());
+                assertEquals(1, newEntry1.updateCounter());
+                assertArrayEquals(key1, newEntry1.key());
+                assertArrayEquals(val11, newEntry1.value());
+            } else if (state == 2) {
+                assertTrue(event.single());
 
-        assertTrue(watchEvent.single());
+                EntryEvent e1 = event.entryEvent();
 
-        EntryEvent e1 = watchEvent.entryEvent();
+                Entry oldEntry1 = e1.oldEntry();
 
-        Entry oldEntry1 = e1.oldEntry();
+                assertFalse(oldEntry1.empty());
+                assertFalse(oldEntry1.tombstone());
+                assertEquals(1, oldEntry1.revision());
+                assertEquals(1, oldEntry1.updateCounter());
+                assertArrayEquals(key1, oldEntry1.key());
+                assertArrayEquals(val11, oldEntry1.value());
 
-        assertTrue(oldEntry1.empty());
-        assertFalse(oldEntry1.tombstone());
+                Entry newEntry1 = e1.newEntry();
 
-        Entry newEntry1 = e1.newEntry();
+                assertFalse(newEntry1.empty());
+                assertFalse(newEntry1.tombstone());
+                assertEquals(3, newEntry1.revision());
+                assertEquals(4, newEntry1.updateCounter());
+                assertArrayEquals(key1, newEntry1.key());
+                assertArrayEquals(val12, newEntry1.value());
+            }
+        });
 
-        assertFalse(newEntry1.empty());
-        assertFalse(newEntry1.tombstone());
-        assertEquals(1, newEntry1.revision());
-        assertEquals(1, newEntry1.updateCounter());
-        assertArrayEquals(key1, newEntry1.key());
-        assertArrayEquals(val11, newEntry1.value());
+        storage.putAll(List.of(key1, key2), List.of(val11, val21));
 
-        assertFalse(it.hasNext());
+        assertEquals(1, storage.revision());
+        assertEquals(2, storage.updateCounter());
 
         storage.put(key2, val22);
-
-        assertFalse(it.hasNext());
-
         storage.put(key1, val12);
 
-        assertTrue(it.hasNext());
-
-        watchEvent = it.next();
-
-        assertTrue(watchEvent.single());
-
-        e1 = watchEvent.entryEvent();
-
-        oldEntry1 = e1.oldEntry();
-
-        assertFalse(oldEntry1.empty());
-        assertFalse(oldEntry1.tombstone());
-        assertEquals(1, oldEntry1.revision());
-        assertEquals(1, oldEntry1.updateCounter());
-        assertArrayEquals(key1, newEntry1.key());
-        assertArrayEquals(val11, newEntry1.value());
-
-        newEntry1 = e1.newEntry();
-
-        assertFalse(newEntry1.empty());
-        assertFalse(newEntry1.tombstone());
-        assertEquals(3, newEntry1.revision());
-        assertEquals(4, newEntry1.updateCounter());
-        assertArrayEquals(key1, newEntry1.key());
-        assertArrayEquals(val12, newEntry1.value());
-
-        assertFalse(it.hasNext());
+        assertThat(awaitFuture, willCompleteSuccessfully());
     }
 
     @Test
-    public void watchCursorForKeySkipNonMatchingEntries() throws Exception {
+    public void watchExactkipNonMatchingEntries() {
         byte[] key1 = key(1);
         byte[] val1v1 = keyValue(1, 11);
         byte[] val1v2 = keyValue(1, 12);
@@ -2340,32 +2299,10 @@ public abstract class AbstractKeyValueStorageTest {
         assertEquals(0, storage.revision());
         assertEquals(0, storage.updateCounter());
 
-        try (Cursor<WatchEvent> cur = storage.watch(key2, 1)) {
-            assertFalse(cur.hasNext());
-            assertThrows(NoSuchElementException.class, cur::next);
-
-            storage.put(key1, val1v1);
-
-            assertFalse(cur.hasNext());
-            assertThrows(NoSuchElementException.class, cur::next);
-
-            storage.put(key1, val1v2);
-
-            assertFalse(cur.hasNext());
-            assertThrows(NoSuchElementException.class, cur::next);
-
-            storage.put(key2, val2);
-
-            assertEquals(3, storage.revision());
-            assertEquals(3, storage.updateCounter());
-
-            assertTrue(cur.hasNext());
-
-            WatchEvent watchEvent = cur.next();
+        CompletableFuture<Void> awaitFuture = watchExact(key2, 1, 1, (event, state) -> {
+            assertTrue(event.single());
 
-            assertTrue(watchEvent.single());
-
-            EntryEvent e1 = watchEvent.entryEvent();
+            EntryEvent e1 = event.entryEvent();
 
             Entry oldEntry1 = e1.oldEntry();
 
@@ -2380,11 +2317,20 @@ public abstract class AbstractKeyValueStorageTest {
             assertEquals(3, newEntry1.updateCounter());
             assertArrayEquals(key2, newEntry1.key());
             assertArrayEquals(val2, newEntry1.value());
-        }
+        });
+
+        storage.put(key1, val1v1);
+        storage.put(key1, val1v2);
+        storage.put(key2, val2);
+
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        assertThat(awaitFuture, willCompleteSuccessfully());
     }
 
     @Test
-    public void watchCursorForKeys() {
+    public void watchExactForKeys() {
         byte[] key1 = key(1);
         byte[] val11 = keyValue(1, 11);
 
@@ -2399,39 +2345,24 @@ public abstract class AbstractKeyValueStorageTest {
         assertEquals(0, storage.revision());
         assertEquals(0, storage.updateCounter());
 
-        Cursor<WatchEvent> cur = storage.watch(List.of(key1, key2), 1);
-
-        Iterator<WatchEvent> it = cur.iterator();
-
-        assertFalse(it.hasNext());
-        assertThrows(NoSuchElementException.class, it::next);
+        CompletableFuture<Void> awaitFuture = watchExact(List.of(key1, key2), 1, 2, (event, state) -> {
+            if (state == 1) {
+                assertFalse(event.single());
+            } else if (state == 2) {
+                assertTrue(event.single());
+            }
+        });
 
         storage.putAll(List.of(key1, key2, key3), List.of(val11, val21, val31));
 
         assertEquals(1, storage.revision());
         assertEquals(3, storage.updateCounter());
 
-        assertTrue(it.hasNext());
-
-        WatchEvent watchEvent = it.next();
-
-        assertFalse(watchEvent.single());
-
-        assertFalse(it.hasNext());
-
         storage.put(key2, val22);
 
-        assertTrue(it.hasNext());
-
-        watchEvent = it.next();
-
-        assertTrue(watchEvent.single());
-
-        assertFalse(it.hasNext());
-
         storage.put(key3, val32);
 
-        assertFalse(it.hasNext());
+        assertThat(awaitFuture, willCompleteSuccessfully());
     }
 
     private static void fill(KeyValueStorage storage, int keySuffix, int num) {
@@ -2447,4 +2378,56 @@ public abstract class AbstractKeyValueStorageTest {
     private static byte[] keyValue(int k, int v) {
         return ("key" + k + '_' + "val" + v).getBytes(UTF_8);
     }
+
+    private CompletableFuture<Void> watchExact(
+            byte[] key, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+    ) {
+        return watch(listener -> storage.watchExact(key, revision, listener), testCondition, expectedNumCalls);
+    }
+
+    private CompletableFuture<Void> watchExact(
+            Collection<byte[]> keys, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+    ) {
+        return watch(listener -> storage.watchExact(keys, revision, listener), testCondition, expectedNumCalls);
+    }
+
+    private CompletableFuture<Void> watchRange(
+            byte[] keyFrom, byte @Nullable [] keyTo, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+    ) {
+        return watch(listener -> storage.watchRange(keyFrom, keyTo, revision, listener), testCondition, expectedNumCalls);
+    }
+
+    private CompletableFuture<Void> watch(
+            Consumer<WatchListener> watchMethod, BiConsumer<WatchEvent, Integer> testCondition, int expectedNumCalls
+    ) {
+        var state = new AtomicInteger();
+
+        var resultFuture = new CompletableFuture<Void>();
+
+        watchMethod.accept(new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent event) {
+                try {
+                    var curState = state.incrementAndGet();
+
+                    testCondition.accept(event, curState);
+
+                    if (curState == expectedNumCalls) {
+                        resultFuture.complete(null);
+                    }
+                } catch (Exception e) {
+                    resultFuture.completeExceptionally(e);
+                }
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                resultFuture.completeExceptionally(e);
+            }
+        });
+
+        storage.startWatches(revision -> {});
+
+        return resultFuture;
+    }
 }
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
index 5c87a65902..530aca3cf8 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
@@ -17,10 +17,22 @@
 
 package org.apache.ignite.internal.metastorage.server;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.nio.file.Path;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
@@ -34,6 +46,62 @@ public class RocksDbKeyValueStorageTest extends AbstractKeyValueStorageTest {
     /** {@inheritDoc} */
     @Override
     KeyValueStorage storage() {
-        return new RocksDbKeyValueStorage(workDir);
+        return new RocksDbKeyValueStorage("test", workDir.resolve("storage"));
+    }
+
+    @Test
+    void testWatchReplayOnSnapshotLoad() throws InterruptedException {
+        storage.put("foo".getBytes(UTF_8), "bar".getBytes(UTF_8));
+        storage.put("baz".getBytes(UTF_8), "quux".getBytes(UTF_8));
+
+        long revisionBeforeSnapshot = storage.revision();
+
+        Path snapshotPath = workDir.resolve("snapshot");
+
+        assertThat(storage.snapshot(snapshotPath), willCompleteSuccessfully());
+
+        storage.close();
+
+        storage = storage();
+
+        storage.start();
+
+        var latch = new CountDownLatch(2);
+
+        storage.watchExact("foo".getBytes(UTF_8), 1, new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent event) {
+                assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(UTF_8)));
+
+                latch.countDown();
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                fail();
+            }
+        });
+
+        storage.watchExact("baz".getBytes(UTF_8), 1, new WatchListener() {
+            @Override
+            public void onUpdate(WatchEvent event) {
+                assertThat(event.entryEvent().newEntry().value(), is("quux".getBytes(UTF_8)));
+
+                latch.countDown();
+            }
+
+            @Override
+            public void onError(Throwable e) {
+                fail();
+            }
+        });
+
+        storage.startWatches(revision -> {});
+
+        storage.restoreSnapshot(snapshotPath);
+
+        assertThat(storage.revision(), is(revisionBeforeSnapshot));
+
+        assertTrue(latch.await(10, TimeUnit.SECONDS));
     }
 }
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
index 5fa0ad41d7..3050c8ff00 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
@@ -24,6 +24,6 @@ class SimpleInMemoryKeyValueStorageTest extends AbstractKeyValueStorageTest {
     /** {@inheritDoc} */
     @Override
     KeyValueStorage storage() {
-        return new SimpleInMemoryKeyValueStorage();
+        return new SimpleInMemoryKeyValueStorage("test");
     }
 }
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java
deleted file mode 100644
index 4d23ca00d2..0000000000
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java
+++ /dev/null
@@ -1,312 +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.internal.metastorage.watch;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
-import org.apache.ignite.lang.ByteArray;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-import org.junit.jupiter.api.Test;
-import org.mockito.ArgumentCaptor;
-
-/**
- * Test class for {@link WatchAggregator}.
- */
-public class WatchAggregatorTest {
-    @Test
-    public void testEventsRouting() {
-        var watchAggregator = new WatchAggregator();
-        var lsnr1 = mock(WatchListener.class);
-        var lsnr2 = mock(WatchListener.class);
-
-        watchAggregator.add(new ByteArray("1"), lsnr1);
-        watchAggregator.add(new ByteArray("2"), lsnr2);
-
-        var entryEvt1 = new EntryEvent(
-                entry("1", "value1", 1, 1),
-                entry("1", "value1n", 1, 1)
-        );
-
-        var entryEvt2 = new EntryEvent(
-                entry("2", "value2", 1, 1),
-                entry("2", "value2n", 1, 1)
-        );
-
-        watchAggregator.watch(1, (v1, v2) -> {
-        }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
-        var watchEvt1Res = ArgumentCaptor.forClass(WatchEvent.class);
-        verify(lsnr1).onUpdate(watchEvt1Res.capture());
-        assertEquals(List.of(entryEvt1), watchEvt1Res.getValue().entryEvents());
-
-        var watchEvt2Res = ArgumentCaptor.forClass(WatchEvent.class);
-        verify(lsnr2).onUpdate(watchEvt2Res.capture());
-        assertEquals(List.of(entryEvt2), watchEvt2Res.getValue().entryEvents());
-    }
-
-    @Test
-    public void testCancel() {
-        var watchAggregator = new WatchAggregator();
-        var lsnr1 = mock(WatchListener.class);
-        when(lsnr1.onUpdate(any())).thenReturn(true);
-        var lsnr2 = mock(WatchListener.class);
-        when(lsnr2.onUpdate(any())).thenReturn(true);
-        final var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
-        var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
-
-        var entryEvt1 = new EntryEvent(
-                entry("1", "value1", 1, 1),
-                entry("1", "value1n", 1, 1)
-        );
-
-        var entryEvt2 = new EntryEvent(
-                entry("2", "value2", 1, 1),
-                entry("2", "value2n", 1, 1)
-        );
-
-        watchAggregator.watch(1, (v1, v2) -> {
-        }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
-        verify(lsnr1, times(1)).onUpdate(any());
-        verify(lsnr2, times(1)).onUpdate(any());
-
-        watchAggregator.cancel(id1);
-        watchAggregator.watch(1, (v1, v2) -> {
-        }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
-        verify(lsnr1, times(1)).onUpdate(any());
-        verify(lsnr2, times(2)).onUpdate(any());
-    }
-
-    @Test
-    public void testCancelByFalseFromListener() {
-        var watchAggregator = new WatchAggregator();
-        var lsnr1 = mock(WatchListener.class);
-        when(lsnr1.onUpdate(any())).thenReturn(false);
-        var lsnr2 = mock(WatchListener.class);
-        when(lsnr2.onUpdate(any())).thenReturn(true);
-        var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
-        var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
-
-        var entryEvt1 = new EntryEvent(
-                entry("1", "value1", 1, 1),
-                entry("1", "value1n", 1, 1)
-        );
-
-        var entryEvt2 = new EntryEvent(
-                entry("2", "value2", 1, 1),
-                entry("2", "value2n", 1, 1)
-        );
-
-        watchAggregator.watch(1, (v1, v2) -> {
-        }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
-        verify(lsnr1, times(1)).onUpdate(any());
-        verify(lsnr2, times(1)).onUpdate(any());
-
-        watchAggregator.watch(1, (v1, v2) -> {
-        }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
-        verify(lsnr1, times(1)).onUpdate(any());
-        verify(lsnr2, times(2)).onUpdate(any());
-
-    }
-
-    @Test
-    public void testOneCriterionInference() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        assertEquals(new KeyCriterion.ExactCriterion(new ByteArray("key")), keyCriterion);
-    }
-
-    @Test
-    public void testTwoExactCriteriaUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key1"), null);
-        watchAggregator.add(new ByteArray("key2"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.CollectionCriterion(
-                new HashSet<>(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")))
-        );
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testTwoEqualExactCriteriaUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key1"), null);
-        watchAggregator.add(new ByteArray("key1"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        assertEquals(keyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testThatKeyCriteriaUnionAssociative() {
-        var data = Arrays.asList(
-                new KeyCriterion.RangeCriterion(new ByteArray("0"), new ByteArray("5")),
-                new KeyCriterion.CollectionCriterion(Arrays.asList(new ByteArray("1"), new ByteArray("2"))),
-                new KeyCriterion.ExactCriterion(new ByteArray("3")));
-
-        for (int i = 0; i < data.size() - 1; i++) {
-            for (int j = i + 1; j < data.size(); j++) {
-                assertEquals(data.get(i).union(data.get(j)), data.get(j).union(data.get(i)));
-            }
-        }
-    }
-
-    @Test
-    public void testTwoEqualCollectionCriteriaUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")), null);
-        watchAggregator.add(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.CollectionCriterion(
-                new HashSet<>(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")))
-        );
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testExactInTheMiddleAndRangeCriteriaOnTheEdgesUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key1"), null);
-        watchAggregator.add(new ByteArray("key0"), new ByteArray("key2"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.RangeCriterion(
-                new ByteArray("key0"), new ByteArray("key2"));
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testHighExactAndLowerRangeCriteriaUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key3"), null);
-        watchAggregator.add(new ByteArray("key0"), new ByteArray("key2"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.RangeCriterion(
-                new ByteArray("key0"), new ByteArray("key4"));
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testNullKeyAsStartOfRangeCriterion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key0"), null);
-        watchAggregator.add(null, new ByteArray("key2"), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.RangeCriterion(
-                null, new ByteArray("key2"));
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testNullKeyAsEndOfRangeCriterion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key3"), null);
-        watchAggregator.add(new ByteArray("key1"), null, null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.RangeCriterion(
-                new ByteArray("key1"), null);
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    @Test
-    public void testAllTypesOfCriteriaUnion() {
-        var watchAggregator = new WatchAggregator();
-
-        watchAggregator.add(new ByteArray("key0"), null);
-        watchAggregator.add(new ByteArray("key1"), new ByteArray("key2"), null);
-        watchAggregator.add(Arrays.asList(new ByteArray("key0"), new ByteArray("key3")), null);
-
-        var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
-        var expKeyCriterion = new KeyCriterion.RangeCriterion(
-                new ByteArray("key0"), new ByteArray("key4"));
-        assertEquals(expKeyCriterion, keyCriterion);
-    }
-
-    private Entry entry(String key, String val, long revision, long updateCntr) {
-        return new Entry() {
-            /** {@inheritDoc} */
-            @Override
-            public @NotNull byte[] key() {
-                return key.getBytes(StandardCharsets.UTF_8);
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public @Nullable byte[] value() {
-                return val.getBytes(StandardCharsets.UTF_8);
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public long revision() {
-                return revision;
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public long updateCounter() {
-                return updateCntr;
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public boolean empty() {
-                return false;
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public boolean tombstone() {
-                return false;
-            }
-        };
-    }
-}
diff --git a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
index 855137d330..15cf0c2a0b 100644
--- a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
+++ b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -26,8 +26,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -35,22 +33,33 @@ import java.util.NoSuchElementException;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.LongConsumer;
 import java.util.function.Predicate;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.EntryEvent;
 import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.dsl.StatementResult;
 import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
 import org.apache.ignite.internal.metastorage.impl.EntryImpl;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.Cursor;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Simple in-memory key/value storage.
+ * Simple in-memory key/value storage for tests.
  */
 public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
+    private static final IgniteLogger LOG = Loggers.forClass(SimpleInMemoryKeyValueStorage.class);
+
     /** Lexicographical comparator. */
     private static final Comparator<byte[]> CMP = Arrays::compareUnsigned;
 
@@ -63,7 +72,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(CMP);
 
     /** Revisions index. Value contains all entries which were modified under particular revision. */
-    private Map<Long, NavigableMap<byte[], Value>> revsIdx = new HashMap<>();
+    private NavigableMap<Long, NavigableMap<byte[], Value>> revsIdx = new TreeMap<>();
 
     /** Revision. Will be incremented for each single-entry or multi-entry update operation. */
     private long rev;
@@ -74,6 +83,20 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     /** All operations are queued on this lock. */
     private final Object mux = new Object();
 
+    private final List<Watch> watches = new CopyOnWriteArrayList<>();
+
+    private boolean areWatchesEnabled = false;
+
+    private LongConsumer revisionCallback;
+
+    private final ExecutorService watchExecutor;
+
+    private final List<Entry> updatedEntries = new ArrayList<>();
+
+    public SimpleInMemoryKeyValueStorage(String nodeName) {
+        this.watchExecutor = Executors.newSingleThreadExecutor(NamedThreadFactory.create(nodeName, "watch-executor", LOG));
+    }
+
     /** {@inheritDoc} */
     @Override
     public void start() {
@@ -104,12 +127,17 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
             doPut(key, value, curRev);
 
-            rev = curRev;
+            updateRevision(curRev);
         }
     }
 
+    private void updateRevision(long newRevision) {
+        rev = newRevision;
+
+        notifyWatches(newRevision);
+    }
+
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Entry getAndPut(byte[] key, byte[] bytes) {
         synchronized (mux) {
@@ -117,7 +145,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
             long lastRev = doPut(key, bytes, curRev);
 
-            rev = curRev;
+            updateRevision(curRev);
 
             // Return previous value.
             return doGetValue(key, lastRev);
@@ -135,7 +163,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
         Collection<Entry> res;
@@ -152,7 +179,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Entry get(byte[] key) {
         synchronized (mux) {
@@ -161,7 +187,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Entry get(byte[] key, long revUpperBound) {
         synchronized (mux) {
@@ -170,14 +195,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Collection<Entry> getAll(List<byte[]> keys) {
         return doGetAll(keys, LATEST_REV);
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Collection<Entry> getAll(List<byte[]> keys, long revUpperBound) {
         return doGetAll(keys, revUpperBound);
@@ -190,13 +213,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
             long curRev = rev + 1;
 
             if (doRemove(key, curRev)) {
-                rev = curRev;
+                updateRevision(curRev);
             }
         }
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Entry getAndRemove(byte[] key) {
         synchronized (mux) {
@@ -237,7 +259,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     /** {@inheritDoc} */
-    @NotNull
     @Override
     public Collection<Entry> getAndRemoveAll(List<byte[]> keys) {
         Collection<Entry> res = new ArrayList<>(keys.size());
@@ -306,7 +327,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
             }
 
             if (modified) {
-                rev = curRev;
+                updateRevision(curRev);
             }
 
             return branch;
@@ -350,7 +371,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                     }
 
                     if (modified) {
-                        rev = curRev;
+                        updateRevision(curRev);
                     }
 
                     return branch.update().result();
@@ -394,25 +415,34 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     @Override
-    public Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev) {
+    public void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener) {
         assert keyFrom != null : "keyFrom couldn't be null.";
         assert rev > 0 : "rev must be positive.";
 
-        return new WatchCursor(rev, k ->
-                CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
-        );
+        Predicate<byte[]> rangePredicate = keyTo == null
+                ? k -> CMP.compare(keyFrom, k) <= 0
+                : k -> CMP.compare(keyFrom, k) <= 0 && CMP.compare(keyTo, k) > 0;
+
+        watches.add(new Watch(rev, listener, rangePredicate));
     }
 
     @Override
-    public Cursor<WatchEvent> watch(byte[] key, long rev) {
+    public void watchPrefix(byte[] prefix, long rev, WatchListener listener) {
+        watchRange(prefix, incrementArray(prefix), rev, listener);
+    }
+
+    @Override
+    public void watchExact(byte[] key, long rev, WatchListener listener) {
         assert key != null : "key couldn't be null.";
         assert rev > 0 : "rev must be positive.";
 
-        return new WatchCursor(rev, k -> CMP.compare(k, key) == 0);
+        Predicate<byte[]> exactPredicate = k -> CMP.compare(k, key) == 0;
+
+        watches.add(new Watch(rev, listener, exactPredicate));
     }
 
     @Override
-    public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+    public void watchExact(Collection<byte[]> keys, long rev, WatchListener listener) {
         assert keys != null && !keys.isEmpty() : "keys couldn't be null or empty: " + keys;
         assert rev > 0 : "rev must be positive.";
 
@@ -420,7 +450,95 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
         keySet.addAll(keys);
 
-        return new WatchCursor(rev, keySet::contains);
+        Predicate<byte[]> inPredicate = keySet::contains;
+
+        watches.add(new Watch(rev, listener, inPredicate));
+    }
+
+    @Override
+    public void startWatches(LongConsumer revisionCallback) {
+        synchronized (mux) {
+            areWatchesEnabled = true;
+
+            assert this.revisionCallback == null;
+
+            this.revisionCallback = revisionCallback;
+
+            replayUpdates();
+        }
+    }
+
+    private void replayUpdates() {
+        long minWatchRevision = watches.stream()
+                .mapToLong(Watch::targetRevision)
+                .min()
+                .orElse(-1);
+
+        if (minWatchRevision == -1) {
+            return;
+        }
+
+        revsIdx.tailMap(minWatchRevision)
+                .forEach((revision, entries) -> {
+                    entries.forEach((key, value) -> {
+                        var entry = new EntryImpl(key, value.bytes(), revision, value.updateCounter());
+
+                        updatedEntries.add(entry);
+                    });
+
+                    notifyWatches(revision);
+                });
+    }
+
+    private void notifyWatches(long revision) {
+        if (!areWatchesEnabled || updatedEntries.isEmpty()) {
+            return;
+        }
+
+        // Make a local copy of the non-volatile field while we are still under the lock.
+        LongConsumer revisionCallback = this.revisionCallback;
+
+        var updatedEntriesCopy = List.copyOf(updatedEntries);
+
+        updatedEntries.clear();
+
+        watchExecutor.execute(() -> {
+            for (Watch watch : watches) {
+                var entryEvents = new ArrayList<EntryEvent>();
+
+                for (Entry newEntry : updatedEntriesCopy) {
+                    byte[] newKey = newEntry.key();
+
+                    if (watch.matches(newKey, revision)) {
+                        Entry oldEntry = get(newKey, revision - 1);
+
+                        entryEvents.add(new EntryEvent(oldEntry, newEntry));
+                    }
+                }
+
+                if (!entryEvents.isEmpty()) {
+                    var event = new WatchEvent(entryEvents, revision);
+
+                    try {
+                        watch.onUpdate(event);
+                    } catch (Exception e) {
+                        watch.onError(e);
+
+                        LOG.error("Error occurred when processing a watch event {}, watch will be disabled", e, event);
+
+                        watches.remove(watch);
+                    }
+                }
+            }
+
+            // Watch processing for the new revision is done, notify the revision callback.
+            revisionCallback.accept(revision);
+        });
+    }
+
+    @Override
+    public void removeWatch(WatchListener listener) {
+        watches.removeIf(watch -> watch.listener() == listener);
     }
 
     @Override
@@ -428,7 +546,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         synchronized (mux) {
             NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(CMP);
 
-            Map<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new HashMap<>();
+            NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new TreeMap<>();
 
             keysIdx.forEach((key, revs) -> compactForKey(key, revs, compactedKeysIdx, compactedRevsIdx));
 
@@ -440,10 +558,9 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
     @Override
     public void close() {
-        // No-op.
+        IgniteUtils.shutdownAndAwaitTermination(watchExecutor, 10, TimeUnit.SECONDS);
     }
 
-    @NotNull
     @Override
     public CompletableFuture<Void> snapshot(Path snapshotPath) {
         throw new UnsupportedOperationException();
@@ -490,7 +607,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         }
     }
 
-    @NotNull
     private Collection<Entry> doGetAll(List<byte[]> keys, long rev) {
         assert keys != null : "keys list can't be null.";
         assert !keys.isEmpty() : "keys list can't be empty.";
@@ -507,7 +623,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return res;
     }
 
-    @NotNull
     private Entry doGet(byte[] key, long revUpperBound) {
         assert revUpperBound >= LATEST_REV : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
 
@@ -537,7 +652,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
      * Returns maximum revision which must be less or equal to {@code upperBoundRev}. If there is no such revision then {@code -1} will be
      * returned.
      *
-     * @param revs          Revisions list.
+     * @param revs Revisions list.
      * @param upperBoundRev Revision upper bound.
      * @return Appropriate revision or {@code -1} if there is no such revision.
      */
@@ -555,7 +670,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return -1;
     }
 
-    @NotNull
     private Entry doGetValue(byte[] key, long lastRev) {
         if (lastRev == 0) {
             return EntryImpl.empty(key);
@@ -602,6 +716,10 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                 }
         );
 
+        var updatedEntry = new EntryImpl(key, val.tombstone() ? null : bytes, curRev, curUpdCntr);
+
+        updatedEntries.add(updatedEntry);
+
         return lastRev;
     }
 
@@ -626,10 +744,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
                 entries.put(key, val);
 
+                updatedEntries.add(new EntryImpl(key, bytes, curRev, curUpdCntr));
+
                 revsIdx.put(curRev, entries);
             }
 
-            rev = curRev;
+            updateRevision(curRev);
 
             return curRev;
         }
@@ -653,12 +773,11 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     private class RangeCursor implements Cursor<Entry> {
         private final byte[] keyFrom;
 
-        private final byte[] keyTo;
+        private final byte @Nullable [] keyTo;
 
         private final long rev;
 
-        private final Iterator<Entry> it;
-
+        @Nullable
         private Entry nextRetEntry;
 
         private byte[] lastRetKey;
@@ -667,148 +786,93 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
         private boolean finished;
 
-        RangeCursor(byte[] keyFrom, byte[] keyTo, long rev, boolean includeTombstones) {
+        RangeCursor(byte[] keyFrom, byte @Nullable [] keyTo, long rev, boolean includeTombstones) {
             this.keyFrom = keyFrom;
             this.keyTo = keyTo;
             this.rev = rev;
             this.includeTombstones = includeTombstones;
-            this.it = createIterator();
         }
 
         /** {@inheritDoc} */
         @Override
         public boolean hasNext() {
-            return it.hasNext();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public Entry next() {
-            return it.next();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public void close() {
-            // No-op.
-        }
-
-        @NotNull
-        Iterator<Entry> createIterator() {
-            return new Iterator<>() {
-                /** {@inheritDoc} */
-                @Override
-                public boolean hasNext() {
-                    synchronized (mux) {
-                        while (true) {
-                            if (finished) {
-                                return false;
-                            }
-
-                            if (nextRetEntry != null) {
-                                return true;
-                            }
-
-                            byte[] key = lastRetKey;
-
-                            while (!finished || nextRetEntry == null) {
-                                Map.Entry<byte[], List<Long>> e =
-                                        key == null ? keysIdx.ceilingEntry(keyFrom) : keysIdx.higherEntry(key);
-
-                                if (e == null) {
-                                    finished = true;
+            synchronized (mux) {
+                while (true) {
+                    if (finished) {
+                        return false;
+                    }
 
-                                    break;
-                                }
+                    if (nextRetEntry != null) {
+                        return true;
+                    }
 
-                                key = e.getKey();
+                    byte[] key = lastRetKey;
 
-                                if (keyTo != null && CMP.compare(key, keyTo) >= 0) {
-                                    finished = true;
+                    while (!finished || nextRetEntry == null) {
+                        Map.Entry<byte[], List<Long>> e =
+                                key == null ? keysIdx.ceilingEntry(keyFrom) : keysIdx.higherEntry(key);
 
-                                    break;
-                                }
+                        if (e == null) {
+                            finished = true;
 
-                                List<Long> revs = e.getValue();
+                            break;
+                        }
 
-                                assert revs != null && !revs.isEmpty() :
-                                        "Revisions should not be empty or null: [revs=" + revs + ']';
+                        key = e.getKey();
 
-                                long lastRev = maxRevision(revs, rev);
+                        if (keyTo != null && CMP.compare(key, keyTo) >= 0) {
+                            finished = true;
 
-                                if (lastRev == -1) {
-                                    continue;
-                                }
+                            break;
+                        }
 
-                                Entry entry = doGetValue(key, lastRev);
+                        List<Long> revs = e.getValue();
 
-                                if (!entry.tombstone() || includeTombstones) {
-                                    assert !entry.empty() : "Iterator should not return empty entry.";
+                        assert revs != null && !revs.isEmpty() :
+                                "Revisions should not be empty or null: [revs=" + revs + ']';
 
-                                    nextRetEntry = entry;
+                        long lastRev = maxRevision(revs, rev);
 
-                                    break;
-                                }
-                            }
+                        if (lastRev == -1) {
+                            continue;
                         }
-                    }
-                }
-
-                /** {@inheritDoc} */
-                @Override
-                public Entry next() {
-                    synchronized (mux) {
-                        while (true) {
-                            if (finished) {
-                                throw new NoSuchElementException();
-                            }
 
-                            if (nextRetEntry != null) {
-                                Entry e = nextRetEntry;
+                        Entry entry = doGetValue(key, lastRev);
 
-                                nextRetEntry = null;
+                        if (!entry.tombstone() || includeTombstones) {
+                            assert !entry.empty() : "Iterator should not return empty entry.";
 
-                                lastRetKey = e.key();
+                            nextRetEntry = entry;
 
-                                return e;
-                            } else {
-                                hasNext();
-                            }
+                            break;
                         }
                     }
                 }
-            };
+            }
         }
-    }
-
-    /**
-     * Extension of {@link Cursor}.
-     */
-    private class WatchCursor implements Cursor<WatchEvent> {
-        private final Predicate<byte[]> predicate;
 
-        private final Iterator<WatchEvent> it;
-
-        private long lastRetRev;
+        /** {@inheritDoc} */
+        @Override
+        public Entry next() {
+            synchronized (mux) {
+                while (true) {
+                    if (finished) {
+                        throw new NoSuchElementException();
+                    }
 
-        private long nextRetRev = -1;
+                    if (nextRetEntry != null) {
+                        Entry e = nextRetEntry;
 
-        WatchCursor(long rev, Predicate<byte[]> predicate) {
-            this.predicate = predicate;
-            this.lastRetRev = rev - 1;
-            this.it = createIterator();
-        }
+                        nextRetEntry = null;
 
-        /** {@inheritDoc} */
-        @Override
-        public boolean hasNext() {
-            return it.hasNext();
-        }
+                        lastRetKey = e.key();
 
-        /** {@inheritDoc} */
-        @Override
-        public WatchEvent next() {
-            return it.next();
+                        return e;
+                    } else {
+                        hasNext();
+                    }
+                }
+            }
         }
 
         /** {@inheritDoc} */
@@ -816,90 +880,5 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         public void close() {
             // No-op.
         }
-
-        @NotNull
-        Iterator<WatchEvent> createIterator() {
-            return new Iterator<>() {
-                /** {@inheritDoc} */
-                @Override
-                public boolean hasNext() {
-                    synchronized (mux) {
-                        if (nextRetRev != -1) {
-                            return true;
-                        }
-
-                        while (true) {
-                            long curRev = lastRetRev + 1;
-
-                            NavigableMap<byte[], Value> entries = revsIdx.get(curRev);
-
-                            if (entries == null) {
-                                return false;
-                            }
-
-                            for (byte[] key : entries.keySet()) {
-                                if (predicate.test(key)) {
-                                    nextRetRev = curRev;
-
-                                    return true;
-                                }
-                            }
-
-                            lastRetRev++;
-                        }
-                    }
-                }
-
-                /** {@inheritDoc} */
-                @Override
-                public WatchEvent next() {
-                    synchronized (mux) {
-                        while (true) {
-                            if (nextRetRev != -1) {
-                                NavigableMap<byte[], Value> entries = revsIdx.get(nextRetRev);
-
-                                if (entries == null) {
-                                    throw new NoSuchElementException();
-                                }
-
-                                List<EntryEvent> evts = new ArrayList<>(entries.size());
-
-                                for (Map.Entry<byte[], Value> e : entries.entrySet()) {
-                                    byte[] key = e.getKey();
-
-                                    Value val = e.getValue();
-
-                                    if (predicate.test(key)) {
-                                        Entry newEntry;
-
-                                        if (val.tombstone()) {
-                                            newEntry = EntryImpl.tombstone(key, nextRetRev, val.updateCounter());
-                                        } else {
-                                            newEntry = new EntryImpl(key, val.bytes(), nextRetRev, val.updateCounter());
-                                        }
-
-                                        Entry oldEntry = doGet(key, nextRetRev - 1);
-
-                                        evts.add(new EntryEvent(oldEntry, newEntry));
-                                    }
-                                }
-
-                                if (evts.isEmpty()) {
-                                    continue;
-                                }
-
-                                lastRetRev = nextRetRev;
-
-                                nextRetRev = -1;
-
-                                return new WatchEvent(evts);
-                            } else if (!hasNext()) {
-                                throw new NoSuchElementException();
-                            }
-                        }
-                    }
-                }
-            };
-        }
     }
 }
diff --git a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
index 731896acde..e482d5c42f 100644
--- a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
+++ b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
@@ -179,10 +179,10 @@ public interface RaftGroupService {
      *
      * <p>This operation is executed on a group leader.
      *
-     * @param learners List of learners.
+     * @param learners Collection of learners.
      * @return A future.
      */
-    CompletableFuture<Void> addLearners(List<Peer> learners);
+    CompletableFuture<Void> addLearners(Collection<Peer> learners);
 
     /**
      * Removes learners.
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
index 3640f6e593..5f41d131c1 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
@@ -363,7 +363,7 @@ public class RaftGroupServiceImpl implements RaftGroupService {
     }
 
     @Override
-    public CompletableFuture<Void> addLearners(List<Peer> learners) {
+    public CompletableFuture<Void> addLearners(Collection<Peer> learners) {
         Peer leader = this.leader;
 
         if (leader == null) {
diff --git a/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java b/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
index 360f317d6f..0e448509e7 100644
--- a/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
+++ b/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.raft.PeersAndLearners;
 import org.apache.ignite.internal.raft.RaftGroupServiceImpl;
 import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
 import org.apache.ignite.internal.replicator.ReplicationGroupId;
 import org.apache.ignite.internal.testframework.WorkDirectory;
@@ -193,7 +194,16 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
         // Set up a raft group service
         RaftGroupService service = prepareRaftGroup(testInfo);
 
-        beforeFollowerStop(service);
+        CompletableFuture<Void> refreshLeaderFuture = service.refreshLeader()
+                .thenCompose(v -> {
+                    if (service.leader() == null) {
+                        return service.refreshLeader();
+                    } else {
+                        return CompletableFuture.completedFuture(null);
+                    }
+                });
+
+        assertThat(refreshLeaderFuture, willCompleteSuccessfully());
 
         // Select any node that is not the leader of the group
         JraftServerImpl toStop = servers.stream()
@@ -201,13 +211,15 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
                 .findAny()
                 .orElseThrow();
 
+        beforeFollowerStop(service, toStop);
+
         var nodeId = new RaftNodeId(raftGroupId(), toStop.localPeers(raftGroupId()).get(0));
 
         // Get the path to that node's raft directory
         Path serverDataPath = toStop.getServerDataPath(nodeId);
 
         // Get the path to that node's RocksDB key-value storage
-        Path dbPath = getListenerPersistencePath(getListener(toStop, raftGroupId()));
+        Path dbPath = getListenerPersistencePath(getListener(toStop, raftGroupId()), toStop);
 
         int stopIdx = servers.indexOf(toStop);
 
@@ -222,7 +234,7 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
         // Create a snapshot of the raft group
         service.snapshot(service.leader()).get();
 
-        afterFollowerStop(service);
+        afterFollowerStop(service, toStop);
 
         // Create another raft snapshot
         service.snapshot(service.leader()).get();
@@ -256,17 +268,19 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
      * Interacts with the raft group before a follower is stopped.
      *
      * @param service Raft group service.
+     * @param server Raft server that is going to be stopped.
      * @throws Exception If failed.
      */
-    public abstract void beforeFollowerStop(RaftGroupService service) throws Exception;
+    public abstract void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception;
 
     /**
      * Interacts with the raft group after a follower is stopped.
      *
      * @param service Raft group service.
+     * @param server Raft server that has been stopped.
      * @throws Exception If failed.
      */
-    public abstract void afterFollowerStop(RaftGroupService service) throws Exception;
+    public abstract void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception;
 
     /**
      * Interacts with a raft group after the leader has captured a snapshot.
@@ -290,9 +304,10 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
      * Returns path to the group's persistence.
      *
      * @param listener Raft group listener.
+     * @param server Raft server, where the listener has been registered.
      * @return Path to the group's persistence.
      */
-    public abstract Path getListenerPersistencePath(T listener);
+    public abstract Path getListenerPersistencePath(T listener, RaftServer server);
 
     /**
      * Creates raft group listener.
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 c82a5bd2c8..5197674999 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
@@ -56,11 +56,11 @@ import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.StaticNodeFinder;
 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;
@@ -138,14 +138,14 @@ public class ItDistributedConfigurationPropertiesTest {
                     clusterService,
                     cmgManager,
                     raftManager,
-                    new SimpleInMemoryKeyValueStorage()
+                    new SimpleInMemoryKeyValueStorage(name())
             );
 
             // create a custom storage implementation that is able to "lose" some storage updates
             var distributedCfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager) {
                 /** {@inheritDoc} */
                 @Override
-                public synchronized void registerConfigurationListener(@NotNull ConfigurationStorageListener listener) {
+                public synchronized void registerConfigurationListener(ConfigurationStorageListener listener) {
                     super.registerConfigurationListener(changedEntries -> {
                         if (receivesUpdates) {
                             return listener.onEntriesChanged(changedEntries);
@@ -168,14 +168,18 @@ public class ItDistributedConfigurationPropertiesTest {
         /**
          * Starts the created components.
          */
-        void start() throws Exception {
+        void start() {
             vaultManager.start();
 
             Stream.of(clusterService, raftManager, cmgManager, metaStorageManager)
                     .forEach(IgniteComponent::start);
 
             // deploy watches to propagate data from the metastore into the vault
-            metaStorageManager.deployWatches();
+            try {
+                metaStorageManager.deployWatches();
+            } catch (NodeStoppingException e) {
+                throw new RuntimeException(e);
+            }
 
             distributedCfgManager.start();
         }
@@ -205,7 +209,7 @@ public class ItDistributedConfigurationPropertiesTest {
         }
 
         String name() {
-            return clusterService.topologyService().localMember().name();
+            return clusterService.localConfiguration().getName();
         }
     }
 
@@ -240,8 +244,7 @@ public class ItDistributedConfigurationPropertiesTest {
                 raftConfiguration
         );
 
-        firstNode.start();
-        secondNode.start();
+        Stream.of(firstNode, secondNode).parallel().forEach(Node::start);
 
         firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(), "cluster");
     }
@@ -285,7 +288,7 @@ public class ItDistributedConfigurationPropertiesTest {
 
         assertThat(firstValue.value(), is("bar"));
         assertThat(directProxy(secondValue).value(), is("bar"));
-        assertTrue(waitForCondition(() -> "bar".equals(secondValue.value()), 100));
+        assertTrue(waitForCondition(() -> "bar".equals(secondValue.value()), 1000));
 
         // disable storage updates on the second node. This way the new values will never be propagated into the
         // configuration storage
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 3ef167c751..7b31d379aa 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
@@ -23,12 +23,12 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutur
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.equalTo;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.Serializable;
 import java.nio.file.Path;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
 import java.util.stream.Stream;
 import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
@@ -112,7 +112,7 @@ public class ItDistributedConfigurationStorageTest {
                     clusterService,
                     cmgManager,
                     raftManager,
-                    new SimpleInMemoryKeyValueStorage()
+                    new SimpleInMemoryKeyValueStorage(name())
             );
 
             cfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager);
@@ -150,7 +150,7 @@ public class ItDistributedConfigurationStorageTest {
         }
 
         String name() {
-            return clusterService.topologyService().localMember().name();
+            return clusterService.localConfiguration().getName();
         }
     }
 
@@ -174,7 +174,7 @@ public class ItDistributedConfigurationStorageTest {
             assertThat(node.cfgStorage.write(data, 0), willBe(equalTo(true)));
             assertThat(node.cfgStorage.writeConfigurationRevision(0, 1), willCompleteSuccessfully());
 
-            waitForCondition(() -> Objects.nonNull(node.vaultManager.get(MetaStorageManagerImpl.APPLIED_REV).join()), 3000);
+            assertTrue(waitForCondition(() -> node.metaStorageManager.appliedRevision() != 0, 3000));
         } finally {
             node.stop();
         }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
index 9cfdb9437e..fed69aedd6 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
@@ -509,7 +509,7 @@ public class ItRebalanceDistributedTest {
                     clusterService,
                     cmgManager,
                     raftManager,
-                    new SimpleInMemoryKeyValueStorage()
+                    new SimpleInMemoryKeyValueStorage(clusterService.localConfiguration().getName())
             );
 
             cfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager);
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
index 2e9709d179..9ba9d228b5 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
@@ -164,7 +164,7 @@ public class ItDataSchemaSyncTest extends IgniteAbstractTest {
 
         IgnitionManager.stop(nodeToStop);
 
-        listenerInhibitor.stopWithoutResend();
+        listenerInhibitor.stopInhibit();
 
         CompletableFuture<Ignite> ignite1Fut = nodesBootstrapCfg.entrySet().stream()
                 .filter(k -> k.getKey().equals(nodeToStop))
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
index cb52ced0c7..1831f1a14f 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.runner.app;
 
 import static org.apache.ignite.internal.recovery.ConfigurationCatchUpListener.CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.apache.ignite.utils.ClusterServiceTestUtils.defaultSerializationRegistry;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -38,6 +39,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.ListIterator;
+import java.util.Objects;
 import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
@@ -48,6 +50,7 @@ import java.util.function.Function;
 import java.util.function.IntFunction;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
+import java.util.stream.Stream;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
 import org.apache.ignite.internal.app.IgniteImpl;
@@ -263,7 +266,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
                 clusterSvc,
                 cmgManager,
                 raftMgr,
-                new RocksDbKeyValueStorage(dir.resolve("metastorage"))
+                new RocksDbKeyValueStorage(name, dir.resolve("metastorage"))
         );
 
         var cfgStorage = new DistributedConfigurationStorage(metaStorageMgr, vault);
@@ -575,7 +578,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
 
                     return (IgniteImpl) future.join();
                 })
-                .collect(Collectors.toUnmodifiableList());
+                .collect(Collectors.toList());
     }
 
     /**
@@ -688,10 +691,10 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * Restarts the node which stores some data.
      */
     @Test
-    public void nodeWithDataTest() {
-        Ignite ignite = startNode(0);
+    public void nodeWithDataTest() throws InterruptedException {
+        IgniteImpl ignite = startNode(0);
 
-        createTableWithData(ignite, TABLE_NAME, 1);
+        createTableWithData(List.of(ignite), TABLE_NAME, 1);
 
         stopNode(0);
 
@@ -704,7 +707,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * Starts two nodes and checks that the data are storing through restarts. Nodes restart in the same order when they started at first.
      */
     @Test
-    public void testTwoNodesRestartDirect() {
+    public void testTwoNodesRestartDirect() throws InterruptedException {
         twoNodesRestart(true);
     }
 
@@ -712,7 +715,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * Starts two nodes and checks that the data are storing through restarts. Nodes restart in reverse order when they started at first.
      */
     @Test
-    public void testTwoNodesRestartReverse() {
+    public void testTwoNodesRestartReverse() throws InterruptedException {
         twoNodesRestart(false);
     }
 
@@ -721,17 +724,17 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      *
      * @param directOrder When the parameter is true, nodes restart in direct order, otherwise they restart in reverse order.
      */
-    private void twoNodesRestart(boolean directOrder) {
-        Ignite ignite = startNode(0);
-
-        startNode(1);
+    private void twoNodesRestart(boolean directOrder) throws InterruptedException {
+        List<IgniteImpl> nodes = startNodes(2);
 
-        createTableWithData(ignite, TABLE_NAME, 2);
-        createTableWithData(ignite, TABLE_NAME_2, 2);
+        createTableWithData(nodes, TABLE_NAME, 2);
+        createTableWithData(nodes, TABLE_NAME_2, 2);
 
         stopNode(0);
         stopNode(1);
 
+        Ignite ignite;
+
         if (directOrder) {
             startNode(0);
             ignite = startNode(1);
@@ -797,12 +800,12 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * checks that the table created before node stop, is not available when majority if lost.
      */
     @Test
-    public void testOneNodeRestartWithGap() {
-        Ignite ignite = startNode(0);
+    public void testOneNodeRestartWithGap() throws InterruptedException {
+        IgniteImpl ignite = startNode(0);
 
         List<IgniteComponent> components = startPartialNode(1, null);
 
-        createTableWithData(ignite, TABLE_NAME, 2);
+        createTableWithData(List.of(ignite), TABLE_NAME, 2);
 
         stopPartialNode(components);
 
@@ -828,12 +831,12 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * Checks that the table created in cluster of 2 nodes, is recovered on a node after restart of this node.
      */
     @Test
-    public void testRecoveryOnOneNode() {
-        Ignite ignite = startNode(0);
+    public void testRecoveryOnOneNode() throws InterruptedException {
+        IgniteImpl ignite = startNode(0);
 
         List<IgniteComponent> components = startPartialNode(1, null);
 
-        createTableWithData(ignite, TABLE_NAME, 2, 1);
+        createTableWithData(List.of(ignite), TABLE_NAME, 2, 1);
 
         stopPartialNode(components);
 
@@ -850,13 +853,11 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      * Checks that a cluster is able to restart when some changes were made in configuration.
      */
     @Test
-    public void testRestartDiffConfig() {
+    public void testRestartDiffConfig() throws InterruptedException {
         List<IgniteImpl> ignites = startNodes(2);
 
-        Ignite ignite0 = ignites.get(0);
-
-        createTableWithData(ignite0, TABLE_NAME, 2);
-        createTableWithData(ignite0, TABLE_NAME_2, 2);
+        createTableWithData(ignites, TABLE_NAME, 2);
+        createTableWithData(ignites, TABLE_NAME_2, 2);
 
         stopNode(0);
         stopNode(1);
@@ -880,17 +881,19 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      */
     @Test
     @WithSystemProperty(key = CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY, value = "0")
-    public void testCfgGapWithoutData() {
+    public void testCfgGapWithoutData() throws InterruptedException {
         List<IgniteImpl> nodes = startNodes(3);
 
-        createTableWithData(nodes.get(0), TABLE_NAME, nodes.size());
+        createTableWithData(nodes, TABLE_NAME, nodes.size());
 
         log.info("Stopping the node.");
 
         stopNode(nodes.size() - 1);
 
-        createTableWithData(nodes.get(0), TABLE_NAME_2, nodes.size());
-        createTableWithData(nodes.get(0), TABLE_NAME_2 + "0", nodes.size());
+        nodes.set(nodes.size() - 1, null);
+
+        createTableWithData(nodes, TABLE_NAME_2, nodes.size());
+        createTableWithData(nodes, TABLE_NAME_2 + "0", nodes.size());
 
         log.info("Starting the node.");
 
@@ -909,7 +912,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      */
     @Test
     @WithSystemProperty(key = CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY, value = "0")
-    public void testMetastorageStop() {
+    public void testMetastorageStop() throws InterruptedException {
         int cfgGap = 4;
 
         List<IgniteImpl> nodes = startNodes(3);
@@ -918,8 +921,10 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
 
         stopNode(nodes.size() - 1);
 
+        nodes.set(nodes.size() - 1, null);
+
         for (int i = 0; i < cfgGap; i++) {
-            createTableWithData(nodes.get(0), "t" + i, nodes.size(), 1);
+            createTableWithData(nodes, "t" + i, nodes.size(), 1);
         }
 
         log.info("Starting the node.");
@@ -956,18 +961,20 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
      */
     @Test
     @Disabled("https://issues.apache.org/jira/browse/IGNITE-17770")
-    public void testCfgGap() {
+    public void testCfgGap() throws InterruptedException {
         List<IgniteImpl> nodes = startNodes(4);
 
-        createTableWithData(nodes.get(0), "t1", nodes.size());
+        createTableWithData(nodes, "t1", nodes.size());
 
         log.info("Stopping the node.");
 
         stopNode(nodes.size() - 1);
 
+        nodes.set(nodes.size() - 1, null);
+
         checkTableWithData(nodes.get(0), "t1");
 
-        createTableWithData(nodes.get(0), "t2", nodes.size());
+        createTableWithData(nodes, "t2", nodes.size());
 
         log.info("Starting the node.");
 
@@ -1001,27 +1008,30 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
     /**
      * Creates a table and load data to it.
      *
-     * @param ignite Ignite.
+     * @param nodes Ignite nodes.
      * @param name Table name.
      * @param replicas Replica factor.
      */
-    private static void createTableWithData(Ignite ignite, String name, int replicas) {
-        createTableWithData(ignite, name, replicas, 10);
+    private void createTableWithData(List<IgniteImpl> nodes, String name, int replicas) throws InterruptedException {
+        createTableWithData(nodes, name, replicas, 10);
     }
 
     /**
      * Creates a table and load data to it.
      *
-     * @param ignite Ignite.
+     * @param nodes Ignite nodes.
      * @param name Table name.
      * @param replicas Replica factor.
      * @param partitions Partitions count.
      */
-    private static void createTableWithData(Ignite ignite, String name, int replicas, int partitions) {
-        try (Session session = ignite.sql().createSession()) {
+    private void createTableWithData(List<IgniteImpl> nodes, String name, int replicas, int partitions)
+            throws InterruptedException {
+        try (Session session = nodes.get(0).sql().createSession()) {
             session.execute(null, "CREATE TABLE " + name
                     + "(id INT PRIMARY KEY, name VARCHAR) WITH replicas=" + replicas + ", partitions=" + partitions);
 
+            waitForIndex(nodes, name + "_PK");
+
             for (int i = 0; i < 100; i++) {
                 session.execute(null, "INSERT INTO " + name + "(id, name) VALUES (?, ?)",
                         i, VALUE_PRODUCER.apply(i));
@@ -1029,6 +1039,37 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
         }
     }
 
+    private void waitForIndex(Collection<IgniteImpl> nodes, String indexName) throws InterruptedException {
+        // FIXME: Wait for the index to be created on all nodes,
+        //  this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the PK index.
+
+        Stream<TablesConfiguration> partialTablesConfiguration = Stream.empty();
+
+        if (partialNode != null) {
+            partialTablesConfiguration = partialNode.stream()
+                    .filter(ConfigurationManager.class::isInstance)
+                    .map(c -> ((ConfigurationManager) c).configurationRegistry().getConfiguration(TablesConfiguration.KEY))
+                    .filter(Objects::nonNull)
+                    .findAny()
+                    .map(Stream::of)
+                    .orElseThrow();
+        }
+
+        Stream<TablesConfiguration> nodesTablesConfigurations = nodes.stream()
+                .filter(Objects::nonNull)
+                .map(node -> node.clusterConfiguration().getConfiguration(TablesConfiguration.KEY));
+
+        List<TablesConfiguration> tablesConfigurations = Stream.concat(nodesTablesConfigurations, partialTablesConfiguration)
+                .collect(Collectors.toList());
+
+        assertTrue(waitForCondition(
+                () -> tablesConfigurations.stream()
+                        .map(cfg -> cfg.indexes().get(indexName.toUpperCase()))
+                        .allMatch(Objects::nonNull),
+                10_000
+        ));
+    }
+
     /**
      * Creates a table.
      *
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
index 13ebf99dad..ba9eb80043 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
@@ -158,7 +158,6 @@ public class ItTablesApiTest extends IgniteAbstractTest {
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18429")
     public void testTableAlreadyCreatedFromLaggedNode() throws Exception {
         clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
 
@@ -395,7 +394,6 @@ public class ItTablesApiTest extends IgniteAbstractTest {
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18429")
     public void testCreateDropTable() throws Exception {
         clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
index 113f4e91b7..e9d0766518 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
@@ -18,14 +18,22 @@
 package org.apache.ignite.internal.sql.engine;
 
 import static org.apache.ignite.internal.sql.engine.util.Commons.IN_BUFFER_SIZE;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
+import java.util.Objects;
 import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
 import org.apache.ignite.table.Table;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
@@ -71,7 +79,7 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
      */
     @ParameterizedTest(name = "tableSize={0}, partitions={1}")
     @MethodSource("rowsWithPartitionsArgs")
-    public void test(int rows, int partitions) {
+    public void test(int rows, int partitions) throws InterruptedException {
         prepareDataSet(rows, partitions);
 
         var res = sql("SELECT /*+ DISABLE_RULE('NestedLoopJoinConverter', 'MergeJoinConverter') */"
@@ -84,7 +92,7 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
         res.forEach(r -> assertThat(r.get(0), is(r.get(1))));
     }
 
-    private void prepareDataSet(int rowsCount, int parts) {
+    private void prepareDataSet(int rowsCount, int parts) throws InterruptedException {
         Object[][] dataRows = new Object[rowsCount][];
 
         for (int i = 0; i < rowsCount; i++) {
@@ -94,10 +102,28 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
         for (String name : List.of("TEST0", "TEST1")) {
             sql(String.format("CREATE TABLE " + name + "(id INT PRIMARY KEY, jid INT, val VARCHAR) WITH replicas=2,partitions=%d", parts));
 
+            waitForIndex(name + "_PK");
+
             // TODO: https://issues.apache.org/jira/browse/IGNITE-17304 uncomment this
             // sql("CREATE INDEX " + name + "_jid_idx ON " + name + "(jid)");
 
             insertData(name, List.of("ID", "JID", "VAL"), dataRows);
         }
     }
+
+    private static void waitForIndex(String indexName) throws InterruptedException {
+        // FIXME: Wait for the index to be created on all nodes,
+        //  this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the PK index.
+        assertTrue(waitForCondition(
+                () -> CLUSTER_NODES.stream().map(node -> getIndexConfiguration(node, indexName)).allMatch(Objects::nonNull),
+                10_000)
+        );
+    }
+
+    private static @Nullable TableIndexConfiguration getIndexConfiguration(Ignite node, String indexName) {
+        return ((IgniteImpl) node).clusterConfiguration()
+                .getConfiguration(TablesConfiguration.KEY)
+                .indexes()
+                .get(indexName.toUpperCase());
+    }
 }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
index 8b2b3e777e..b2498853ba 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
@@ -23,9 +23,17 @@ import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIn
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsSubPlan;
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTableScan;
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsUnion;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.hamcrest.Matchers.not;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
@@ -46,12 +54,16 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
      * Before all.
      */
     @BeforeAll
-    static void initTestData() {
+    static void initTestData() throws InterruptedException {
         sql("CREATE TABLE developer (id INT PRIMARY KEY, name VARCHAR, depid INT, city VARCHAR, age INT)");
         sql("CREATE INDEX " + DEPID_IDX + " ON developer (depid)");
         sql("CREATE INDEX " + NAME_CITY_IDX + " ON developer (name DESC, city DESC)");
         sql("CREATE INDEX " + NAME_DEPID_CITY_IDX + " ON developer (name DESC, depid DESC, city DESC)");
 
+        waitForIndex(DEPID_IDX);
+        waitForIndex(NAME_CITY_IDX);
+        waitForIndex(NAME_DEPID_CITY_IDX);
+
         insertData("DEVELOPER", List.of("ID", "NAME", "DEPID", "CITY", "AGE"), new Object[][]{
                 {1, "Mozart", 3, "Vienna", 33},
                 {2, "Beethoven", 2, "Vienna", 44},
@@ -81,6 +93,8 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
         sql("CREATE TABLE unwrap_pk(f1 VARCHAR, f2 BIGINT, f3 BIGINT, f4 BIGINT, primary key(f2, f1))");
         sql("CREATE INDEX " + PK_SORTED_IDX + " ON unwrap_pk(f2, f1)");
 
+        waitForIndex(PK_SORTED_IDX);
+
         insertData("UNWRAP_PK", List.of("F1", "F2", "F3", "F4"), new Object[][]{
                 {"Petr", 1L, 2L, 3L},
                 {"Ivan", 2L, 2L, 4L},
@@ -94,6 +108,8 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
         sql("CREATE TABLE t1 (id INT PRIMARY KEY, val INT)");
         sql("CREATE INDEX t1_idx on t1(val DESC)");
 
+        waitForIndex("t1_idx");
+
         insertData("T1", List.of("ID", "VAL"), new Object[][]{
                 {1, null},
                 {2, null},
@@ -105,6 +121,22 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
         });
     }
 
+    private static void waitForIndex(String indexName) throws InterruptedException {
+        // FIXME: Wait for the sorted index to be created on all nodes,
+        //  this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the sorted index.
+        assertTrue(waitForCondition(
+                () -> CLUSTER_NODES.stream().map(node -> getSortedIndexConfig(node, indexName)).allMatch(Objects::nonNull),
+                10_000)
+        );
+    }
+
+    private static @Nullable TableIndexConfiguration getSortedIndexConfig(Ignite node, String indexName) {
+        return ((IgniteImpl) node).clusterConfiguration()
+                .getConfiguration(TablesConfiguration.KEY)
+                .indexes()
+                .get(indexName.toUpperCase());
+    }
+
     @Test
     public void testEqualsFilterWithUnwrpKey() {
         assertQuery("SELECT F1 FROM UNWRAP_PK WHERE F2=2")
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
index a9ed2853a0..e8e59e8acb 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.table;
 
 import static org.apache.ignite.internal.index.SortedIndex.INCLUDE_LEFT;
 import static org.apache.ignite.internal.index.SortedIndex.INCLUDE_RIGHT;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -30,6 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Flow.Publisher;
@@ -39,6 +41,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -50,6 +53,7 @@ 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.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
 import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.schema.row.RowAssembler;
 import org.apache.ignite.internal.sql.engine.AbstractBasicIntegrationTest;
@@ -59,6 +63,7 @@ import org.apache.ignite.lang.IgniteStringFormatter;
 import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.Tuple;
 import org.apache.ignite.tx.IgniteTransactions;
+import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
@@ -87,9 +92,16 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
     );
 
     @BeforeEach
-    public void beforeTest() {
+    public void beforeTest() throws InterruptedException {
         TableImpl table = getOrCreateTable();
 
+        // FIXME: Wait for the sorted index to be created on all nodes,
+        //  this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the sorted index.
+        assertTrue(waitForCondition(
+                () -> CLUSTER_NODES.stream().map(ItTableScanTest::getSortedIndexConfig).allMatch(Objects::nonNull),
+                10_000)
+        );
+
         loadData(table);
     }
 
@@ -120,7 +132,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(2);
 
-        IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 2, 10_000);
+        waitForCondition(() -> scannedRows.size() == 2, 10_000);
 
         assertEquals(2, scannedRows.size());
         assertFalse(scanned.isDone());
@@ -162,7 +174,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(1);
 
-        IgniteTestUtils.waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
+        waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
 
         assertEquals(1, scannedRows.size());
         assertFalse(scanned.isDone());
@@ -406,7 +418,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(1);
 
-        IgniteTestUtils.waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
+        waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
 
         assertEquals(1, scannedRows.size());
         assertFalse(scanned.isDone());
@@ -417,7 +429,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(2);
 
-        IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 3, 10_000);
+        waitForCondition(() -> scannedRows.size() == 3, 10_000);
 
         assertEquals(3, scannedRows.size());
         assertFalse(scanned.isDone());
@@ -462,7 +474,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(3);
 
-        IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 3, 10_000);
+        waitForCondition(() -> scannedRows.size() == 3, 10_000);
 
         assertEquals(3, scannedRows.size());
         assertFalse(scanned.isDone());
@@ -597,7 +609,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
 
         subscription.request(1_000); // Request so much entries here to close the publisher.
 
-        IgniteTestUtils.waitForCondition(() -> scanned.isDone(), 10_000);
+        waitForCondition(() -> scanned.isDone(), 10_000);
 
         return scannedRows;
     }
@@ -638,8 +650,14 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
      * @return Index id.
      */
     private static UUID getSortedIndexId() {
-        return ((IgniteImpl) CLUSTER_NODES.get(0)).clusterConfiguration().getConfiguration(TablesConfiguration.KEY).indexes()
-                .get(SORTED_IDX.toUpperCase()).id().value();
+        return getSortedIndexConfig(CLUSTER_NODES.get(0)).id().value();
+    }
+
+    private static @Nullable TableIndexConfiguration getSortedIndexConfig(Ignite node) {
+        return ((IgniteImpl) node).clusterConfiguration()
+                .getConfiguration(TablesConfiguration.KEY)
+                .indexes()
+                .get(SORTED_IDX.toUpperCase());
     }
 
     /**
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
index 3d06f9c8f9..ce220e5af8 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
@@ -17,144 +17,82 @@
 
 package org.apache.ignite.internal.test;
 
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.Mockito.doAnswer;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.getFieldValue;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Optional;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.metastorage.WatchEvent;
 import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
-import org.apache.ignite.internal.metastorage.watch.AggregatedWatch;
-import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
-import org.apache.ignite.internal.testframework.IgniteTestUtils;
-import org.jetbrains.annotations.NotNull;
-import org.junit.platform.commons.util.ReflectionUtils;
-import org.mockito.Mockito;
+import org.apache.ignite.internal.metastorage.server.Watch;
+import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
 
 /**
  * Listener which wraps another one to inhibit events.
  */
-public class WatchListenerInhibitor implements WatchListener {
-    /** Inhibited events. Guarded by {@code this}. */
-    private final Collection<WatchEvent> inhibitEvents = new ArrayList<>();
+public class WatchListenerInhibitor {
+    /** "watches" field captured from the {@link RocksDbKeyValueStorage} instance. */
+    private final List<Watch> watches;
 
-    /** Inhibit flag. Guarded by {@code this}. */
-    private boolean inhibit = false;
-
-    /** Wrapped listener. Guarded by {@code this}. */
-    private WatchListener realListener;
+    /** Latch used to block the watch notification thread. */
+    private final CountDownLatch inhibitLatch = new CountDownLatch(1);
 
     /**
      * Creates the specific listener which can inhibit events for real metastorage listener.
      *
      * @param ignite Ignite.
      * @return Listener inhibitor.
-     * @throws Exception If something wrong when creating the listener inhibitor.
      */
-    public static WatchListenerInhibitor metastorageEventsInhibitor(Ignite ignite)
-            throws Exception {
+    public static WatchListenerInhibitor metastorageEventsInhibitor(Ignite ignite) {
         //TODO: IGNITE-15723 After a component factory will be implemented, need to got rid of reflection here.
-        MetaStorageManagerImpl metaMngr = (MetaStorageManagerImpl) ReflectionUtils.tryToReadFieldValue(
-                IgniteImpl.class,
-                "metaStorageMgr",
-                (IgniteImpl) ignite
-        ).get();
-
-        assertNotNull(metaMngr);
-
-        WatchAggregator aggregator = (WatchAggregator) ReflectionUtils.tryToReadFieldValue(
-                MetaStorageManagerImpl.class,
-                "watchAggregator",
-                metaMngr
-        ).get();
-
-        assertNotNull(aggregator);
-
-        WatchAggregator aggregatorSpy = Mockito.spy(aggregator);
-
-        WatchListenerInhibitor inhibitor = new WatchListenerInhibitor();
-
-        doAnswer(mock -> {
-            Optional<AggregatedWatch> op = (Optional<AggregatedWatch>) mock.callRealMethod();
-
-            assertTrue(op.isPresent());
+        var metaStorageManager = (MetaStorageManagerImpl) getFieldValue(ignite, IgniteImpl.class, "metaStorageMgr");
 
-            inhibitor.setRealListener(op.get().listener());
+        var storage = (RocksDbKeyValueStorage) getFieldValue(metaStorageManager, MetaStorageManagerImpl.class, "storage");
 
-            return Optional.of(new AggregatedWatch(op.get().keyCriterion(), op.get().revision(),
-                    inhibitor));
-        }).when(aggregatorSpy).watch(anyLong(), any());
+        var watches = (List<Watch>) getFieldValue(storage, RocksDbKeyValueStorage.class, "watches");
 
-        IgniteTestUtils.setFieldValue(metaMngr, "watchAggregator", aggregatorSpy);
-
-        // Redeploy metastorage watch. The Watch inhibitor will be used after.
-        metaMngr.unregisterWatch(-1);
-
-        return inhibitor;
+        return new WatchListenerInhibitor(watches);
     }
 
-    /**
-     * Default constructor.
-     */
-    private WatchListenerInhibitor() {
+    private WatchListenerInhibitor(List<Watch> watches) {
+        this.watches = watches;
     }
 
     /**
-     * Sets a wrapped listener.
-     *
-     * @param realListener Listener to wrap.
+     * Starts inhibiting events.
      */
-    private synchronized void setRealListener(WatchListener realListener) {
-        this.realListener = realListener;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized boolean onUpdate(@NotNull WatchEvent evt) {
-        if (!inhibit) {
-            return realListener.onUpdate(evt);
-        }
-
-        return inhibitEvents.add(evt);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void onError(@NotNull Throwable e) {
-        realListener.onError(e);
-    }
-
-    /**
-     * Starts inhibit events.
-     */
-    public synchronized void startInhibit() {
-        inhibit = true;
+    public void startInhibit() {
+        // Inject a watch that matches all keys and revisions and blocks the watch notification thread until the latch is released.
+        var blockingWatch = new Watch(
+                0,
+                new WatchListener() {
+                    @Override
+                    public void onUpdate(WatchEvent event) {
+                        try {
+                            inhibitLatch.await();
+                        } catch (InterruptedException e) {
+                            Thread.currentThread().interrupt();
+
+                            throw new RuntimeException(e);
+                        }
+                    }
+
+                    @Override
+                    public void onError(Throwable e) {
+                    }
+                },
+                key -> true
+        );
+
+        watches.add(0, blockingWatch);
     }
 
     /**
-     * Stops inhibit events.
+     * Stops inhibiting events.
      */
-    public synchronized void stopInhibit() {
-        inhibit = false;
-
-        for (WatchEvent evt : inhibitEvents) {
-            realListener.onUpdate(evt);
-        }
-
-        inhibitEvents.clear();
-    }
-
-    /**
-     * Stops silently, no events resend.
-     */
-    public synchronized void stopWithoutResend() {
-        inhibit = false;
-
-        inhibitEvents.clear();
+    public void stopInhibit() {
+        inhibitLatch.countDown();
     }
 }
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 e9d936e4ce..6d40dc0e95 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
@@ -342,7 +342,7 @@ public class IgniteImpl implements Ignite {
                 clusterSvc,
                 cmgMgr,
                 raftMgr,
-                new RocksDbKeyValueStorage(workDir.resolve(METASTORAGE_DB_PATH))
+                new RocksDbKeyValueStorage(name, workDir.resolve(METASTORAGE_DB_PATH))
         );
 
         placementDriverMgr = new PlacementDriverManager(metaStorageMgr);
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
index 5c3b27e9d9..3964041b1f 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -46,7 +47,6 @@ import org.apache.ignite.internal.metastorage.dsl.Conditions;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.dsl.Operations;
 import org.apache.ignite.internal.metastorage.dsl.SimpleCondition;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
 import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.util.Cursor;
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.vault.VaultEntry;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -108,12 +107,11 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
      * revision of {@link #MASTER_KEY}.
      *
      * <p>This is true for all cases except for node restart. Key-specific revision values are lost on local vault copy after restart, so
-     * stored {@link MetaStorageManagerImpl#APPLIED_REV} value is used instead. This fact has very important side effect: it's no longer
+     * stored {@link MetaStorageManager#appliedRevision} value is used instead. This fact has very important side effect: it's no longer
      * possible to use {@link ConditionType#REV_EQUAL} on {@link #MASTER_KEY}
      * in {@link DistributedConfigurationStorage#write(Map, long)}. {@link ConditionType#REV_LESS_OR_EQUAL} must be used instead.
      *
      * @see #MASTER_KEY
-     * @see MetaStorageManagerImpl#APPLIED_REV
      * @see #write(Map, long)
      */
     private final AtomicLong changeId = new AtomicLong(0L);
@@ -196,8 +194,8 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<Data> readDataOnRecovery() throws StorageException {
-        CompletableFuture<Data> future = vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
-                .thenCombine(vaultMgr.get(CONFIGURATION_REVISIONS_KEY), this::resolveRevision)
+        CompletableFuture<Data> future = vaultMgr.get(CONFIGURATION_REVISIONS_KEY)
+                .thenApply(configurationRevision -> resolveRevision(metaStorageMgr.appliedRevision(), configurationRevision))
                 .thenApplyAsync(this::readDataOnRecovery0, threadPool);
 
         return registerFuture(future);
@@ -207,13 +205,11 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
      * Resolves current configuration revision based on the saved in the Vault revision of the metastorage and also
      * previous and current revisions of the configuration saved in the Vault.
      *
-     * @param appliedRevEntry Applied revision entry.
+     * @param metaStorageRevision Meta Storage revision.
      * @param revisionsEntry Configuration revisions entry.
      * @return Configuration revision.
      */
-    private long resolveRevision(@Nullable VaultEntry appliedRevEntry, @Nullable VaultEntry revisionsEntry) {
-        long appliedRevision = appliedRevEntry == null ? 0L : ByteUtils.bytesToLong(appliedRevEntry.value());
-
+    private static long resolveRevision(long metaStorageRevision, @Nullable VaultEntry revisionsEntry) {
         if (revisionsEntry != null) {
             byte[] value = revisionsEntry.value();
             long prevMasterKeyRevision = ByteUtils.bytesToLong(value, 0);
@@ -221,7 +217,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
 
             // If current master key revision is higher than applied revision, then node failed
             // before applied revision changed, so we have to use previous master key revision
-            return curMasterKeyRevision <= appliedRevision ? curMasterKeyRevision : prevMasterKeyRevision;
+            return curMasterKeyRevision <= metaStorageRevision ? curMasterKeyRevision : prevMasterKeyRevision;
         } else {
             // Configuration has not been updated yet, so it is safe to return 0 as the revision for the master key.
             return 0L;
@@ -302,8 +298,10 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
             // TODO: properly handle such cases https://issues.apache.org/jira/browse/IGNITE-14604
             metaStorageMgr.registerPrefixWatch(DST_KEYS_START_RANGE, new WatchListener() {
                 @Override
-                public boolean onUpdate(@NotNull WatchEvent events) {
-                    Map<String, Serializable> data = new HashMap<>();
+                public void onUpdate(WatchEvent events) {
+                    var data = new HashMap<String, Serializable>();
+
+                    var vaultData = new HashMap<ByteArray, byte[]>();
 
                     Entry masterKeyEntry = null;
 
@@ -319,6 +317,8 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
 
                             data.put(key, value);
                         }
+
+                        vaultData.put(new ByteArray(e.key()), e.value());
                     }
 
                     // Contract of meta storage ensures that all updates of one revision will come in one batch.
@@ -332,13 +332,21 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
 
                     changeId.set(newChangeId);
 
-                    lsnr.onEntriesChanged(new Data(data, newChangeId)).join();
+                    try {
+                        vaultMgr.putAll(vaultData)
+                                .thenCompose(v -> lsnr.onEntriesChanged(new Data(data, newChangeId)))
+                                .get();
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
 
-                    return true;
+                        throw new StorageException("Interrupted when processing Meta Storage watch", e);
+                    } catch (ExecutionException e) {
+                        throw new StorageException("Error when processing Meta Storage watch", e);
+                    }
                 }
 
                 @Override
-                public void onError(@NotNull Throwable e) {
+                public void onError(Throwable e) {
                     // TODO: need to handle this case and there should some mechanism for registering new watch as far as
                     // TODO: onError unregisters failed watch https://issues.apache.org/jira/browse/IGNITE-14604
                     LOG.warn("Meta storage listener issue", e);
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
index c7b17fb09a..4ef2cc87aa 100644
--- a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
@@ -21,6 +21,7 @@ import static org.apache.ignite.configuration.annotation.ConfigurationType.DISTR
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -42,8 +43,6 @@ import org.apache.ignite.internal.metastorage.WatchEvent;
 import org.apache.ignite.internal.metastorage.WatchListener;
 import org.apache.ignite.internal.metastorage.dsl.Operation;
 import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
-import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
 import org.apache.ignite.lang.ByteArray;
@@ -124,7 +123,7 @@ public class DistributedConfigurationCatchUpTest {
         vaultManager.put(MetaStorageMockWrapper.TEST_KEY, new byte[]{4, 1, 2, 3, 4}).get();
 
         // This emulates a change in MetaStorage that is not related to the configuration.
-        vaultManager.put(MetaStorageManagerImpl.APPLIED_REV, ByteUtils.longToBytes(2)).get();
+        when(wrapper.mock.appliedRevision()).thenReturn(1L);
 
         storage = storage(wrapper);
 
@@ -196,11 +195,11 @@ public class DistributedConfigurationCatchUpTest {
                     .then(invocation -> triggerStorageListener());
 
             // This captures the listener.
-            when(mock.registerPrefixWatch(any(), any())).then(invocation -> {
+            doAnswer(invocation -> {
                 lsnr = invocation.getArgument(1);
 
-                return CompletableFuture.completedFuture(null);
-            });
+                return null;
+            }).when(mock).registerPrefixWatch(any(), any());
         }
 
         /**
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
index 1b76c29845..a4580144ff 100644
--- a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
@@ -45,7 +45,7 @@ import org.junit.jupiter.api.BeforeEach;
 public class DistributedConfigurationStorageTest extends ConfigurationStorageTest {
     private final VaultManager vaultManager = new VaultManager(new InMemoryVaultService());
 
-    private final KeyValueStorage metaStorage = new SimpleInMemoryKeyValueStorage();
+    private final KeyValueStorage metaStorage = new SimpleInMemoryKeyValueStorage("test");
 
     private final MetaStorageManager metaStorageManager = mockMetaStorageManager();
 
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 55863ca962..7c52a4325a 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
@@ -38,6 +38,7 @@ import java.util.function.Function;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridClockImpl;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.raft.server.RaftServer;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
 import org.apache.ignite.internal.raft.service.ItAbstractListenerSnapshotTest;
 import org.apache.ignite.internal.raft.service.RaftGroupListener;
@@ -118,7 +119,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
     /** {@inheritDoc} */
     @Override
-    public void beforeFollowerStop(RaftGroupService service) throws Exception {
+    public void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
         // TODO: https://issues.apache.org/jira/browse/IGNITE-17817 Use Replica layer with new transaction protocol.
         TxManagerImpl txManager = new TxManagerImpl(replicaService, new HeapLockManager(), new HybridClockImpl());
 
@@ -144,7 +145,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
     /** {@inheritDoc} */
     @Override
-    public void afterFollowerStop(RaftGroupService service) throws Exception {
+    public void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
         // TODO: https://issues.apache.org/jira/browse/IGNITE-17817 Use Replica layer with new transaction protocol.
         TxManagerImpl txManager = new TxManagerImpl(replicaService, new HeapLockManager(), new HybridClockImpl());
 
@@ -252,7 +253,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
     /** {@inheritDoc} */
     @Override
-    public Path getListenerPersistencePath(PartitionListener listener) {
+    public Path getListenerPersistencePath(PartitionListener listener, RaftServer server) {
         return paths.get(listener);
     }
 
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 b6b438ef80..4a92a73f98 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
@@ -1765,7 +1765,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private void registerRebalanceListeners() {
         metaStorageMgr.registerPrefixWatch(ByteArray.fromString(PENDING_ASSIGNMENTS_PREFIX), new WatchListener() {
             @Override
-            public boolean onUpdate(@NotNull WatchEvent evt) {
+            public void onUpdate(WatchEvent evt) {
                 if (!busyLock.enterBusy()) {
                     throw new IgniteInternalException(new NodeStoppingException());
                 }
@@ -1776,7 +1776,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     Entry pendingAssignmentsWatchEvent = evt.entryEvent().newEntry();
 
                     if (pendingAssignmentsWatchEvent.value() == null) {
-                        return true;
+                        return;
                     }
 
                     int partId = extractPartitionNumber(pendingAssignmentsWatchEvent.key());
@@ -1901,7 +1901,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     // Do not change peers of the raft group if this is a stale event.
                     // Note that we start raft node before for the sake of the consistency in a starting and stopping raft nodes.
                     if (pendingAssignmentsWatchEvent.revision() < pendingAssignmentsEntry.revision()) {
-                        return true;
+                        return;
                     }
 
                     RaftGroupService partGrpSvc = internalTable.partitionRaftGroupService(partId);
@@ -1916,22 +1916,20 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
                         partGrpSvc.changePeersAsync(pendingConfiguration, leaderWithTerm.term()).join();
                     }
-
-                    return true;
                 } finally {
                     busyLock.leaveBusy();
                 }
             }
 
             @Override
-            public void onError(@NotNull Throwable e) {
+            public void onError(Throwable e) {
                 LOG.warn("Unable to process pending assignments event", e);
             }
         });
 
         metaStorageMgr.registerPrefixWatch(ByteArray.fromString(STABLE_ASSIGNMENTS_PREFIX), new WatchListener() {
             @Override
-            public boolean onUpdate(@NotNull WatchEvent evt) {
+            public void onUpdate(WatchEvent evt) {
                 if (!busyLock.enterBusy()) {
                     throw new IgniteInternalException(new NodeStoppingException());
                 }
@@ -1942,7 +1940,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     Entry stableAssignmentsWatchEvent = evt.entryEvent().newEntry();
 
                     if (stableAssignmentsWatchEvent.value() == null) {
-                        return true;
+                        return;
                     }
 
                     int part = extractPartitionNumber(stableAssignmentsWatchEvent.key());
@@ -1973,22 +1971,20 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                             // no-op
                         }
                     }
-
-                    return true;
                 } finally {
                     busyLock.leaveBusy();
                 }
             }
 
             @Override
-            public void onError(@NotNull Throwable e) {
+            public void onError(Throwable e) {
                 LOG.warn("Unable to process stable assignments event", e);
             }
         });
 
         metaStorageMgr.registerPrefixWatch(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
             @Override
-            public boolean onUpdate(@NotNull WatchEvent evt) {
+            public void onUpdate(WatchEvent evt) {
                 byte[] key = evt.entryEvent().newEntry().key();
 
                 int partitionNumber = extractPartitionNumber(key);
@@ -2008,12 +2004,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                         replicaGrpId,
                         evt
                 );
-
-                return true;
             }
 
             @Override
-            public void onError(@NotNull Throwable e) {
+            public void onError(Throwable e) {
                 LOG.warn("Unable to process switch reduce event", e);
             }
         });