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

[ignite-3] 01/01: IGNITE-14389 Added in-memory meta storage implementation.

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

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

commit 479ea87913ca6e46911046222e2144c92b6e1f2e
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Tue Mar 30 21:21:07 2021 +0300

    IGNITE-14389 Added in-memory meta storage implementation.
---
 .../ignite/internal/affinity/AffinityManager.java  |   34 +-
 .../org/apache/ignite/internal/util}/Cursor.java   |   10 +-
 .../java/org/apache/ignite/lang/ByteArray.java     |   17 +-
 modules/metastorage-client/pom.xml                 |    6 +
 .../client/ITMetaStorageServiceTest.java           |  907 +++++----
 .../internal/metastorage/client/CursorImpl.java    |   84 +-
 .../internal/metastorage/client/EntryImpl.java}    |   73 +-
 .../metastorage/client/MetaStorageServiceImpl.java |  268 ++-
 .../metastorage/client}/CompactedException.java    |    2 +-
 .../ignite/metastorage/client/Condition.java       |  347 ++++
 .../ignite/metastorage/client}/Conditions.java     |   48 +-
 .../apache/ignite/metastorage/client}/Entry.java   |   19 +-
 .../ignite/metastorage/client/EntryEvent.java}     |   24 +-
 .../org/apache/ignite/metastorage/client}/Key.java |    2 +-
 .../metastorage/client/MetaStorageService.java     |  123 +-
 .../ignite/metastorage/client/Operation.java       |  166 ++
 .../client}/OperationTimeoutException.java         |    2 +-
 .../ignite/metastorage/client}/Operations.java     |   14 +-
 .../ignite/metastorage/client/WatchEvent.java      |   82 +
 .../ignite/metastorage/client}/WatchListener.java  |    6 +-
 .../internal/metastorage/common/ConditionType.java |   53 +
 .../metastorage/common/OperationType.java}         |   17 +-
 .../metastorage/common/command/ConditionInfo.java  |   89 +
 .../metastorage/common/command/ErrorResponse.java} |   33 +-
 .../metastorage/common/command/GetAllCommand.java  |   34 +-
 .../common/command/GetAndPutAllCommand.java        |   28 +-
 .../common/command/GetAndPutCommand.java           |   13 +-
 .../common/command/GetAndRemoveAllCommand.java     |   23 +-
 .../common/command/GetAndRemoveCommand.java        |   13 +-
 .../metastorage/common/command/GetCommand.java     |   22 +-
 .../metastorage/common/command/InvokeCommand.java  |   75 +
 ...moveCommand.java => MultipleEntryResponse.java} |   30 +-
 .../{PutCommand.java => OperationInfo.java}        |   54 +-
 .../metastorage/common/command/PutAllCommand.java  |   51 +-
 .../metastorage/common/command/PutCommand.java     |   13 +-
 .../metastorage/common/command/RangeCommand.java   |   29 +-
 .../common/command/RemoveAllCommand.java           |   27 +-
 .../metastorage/common/command/RemoveCommand.java  |   13 +-
 .../common/command/SingleEntryResponse.java        |   92 +
 .../common/command/WatchExactKeysCommand.java      |   29 +-
 .../common/command/WatchRangeKeysCommand.java      |   30 +-
 .../common/command/cursor/CursorCloseCommand.java  |    3 +-
 .../command/cursor/CursorHasNextCommand.java       |    3 +-
 .../common/command/cursor/CursorNextCommand.java   |    3 +-
 .../ignite/metastorage/common/Condition.java       |  370 ----
 .../ignite/metastorage/common/KeyValueStorage.java |   91 -
 .../metastorage/common/KeyValueStorageImpl.java    |  183 --
 .../ignite/metastorage/common/Operation.java       |   93 -
 modules/metastorage-server/pom.xml                 |   64 +
 .../metastorage/server/AbstractCondition.java}     |   27 +-
 .../metastorage/server}/CompactedException.java    |    2 +-
 .../internal/metastorage/server/Condition.java}    |   26 +-
 .../ignite/internal/metastorage/server/Entry.java  |  190 ++
 .../internal/metastorage/server/EntryEvent.java}   |   44 +-
 .../metastorage/server/ExistenceCondition.java     |   74 +
 .../metastorage/server/KeyValueStorage.java        |  206 +++
 .../internal/metastorage/server/Operation.java     |  100 +
 .../metastorage/server/RevisionCondition.java      |  110 ++
 .../server/SimpleInMemoryKeyValueStorage.java      |  775 ++++++++
 .../ignite/internal/metastorage/server/Value.java} |   55 +-
 .../metastorage/server/ValueCondition.java         |   82 +
 .../internal/metastorage/server/WatchEvent.java    |   72 +
 .../server}/raft/MetaStorageCommandListener.java   |  229 ++-
 .../metastorage/server/ExistenceConditionTest.java |   71 +
 .../metastorage/server/RevisionConditionTest.java  |  113 ++
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 1945 ++++++++++++++++++++
 .../metastorage/server/ValueConditionTest.java     |   60 +
 modules/metastorage/pom.xml                        |    4 +-
 .../internal/metastorage/MetaStorageManager.java   |  173 +-
 .../metastorage/watch/AggregatedWatch.java         |    2 +-
 .../internal/metastorage/watch/KeyCriterion.java   |   44 +-
 .../metastorage/watch/WatchAggregator.java         |   49 +-
 .../internal/metastorage/WatchAggregatorTest.java  |  102 +-
 .../apache/ignite/internal/app/IgnitionImpl.java   |    5 +-
 .../storage/DistributedConfigurationStorage.java   |   52 +-
 .../ignite/internal/schema/SchemaManager.java      |   47 +-
 .../internal/table/distributed/TableManager.java   |   14 +-
 parent/pom.xml                                     |    6 +
 pom.xml                                            |    1 +
 79 files changed, 6521 insertions(+), 1971 deletions(-)

diff --git a/modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityManager.java b/modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityManager.java
index 6572a43..57f795a 100644
--- a/modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityManager.java
+++ b/modules/affinity/src/main/java/org/apache/ignite/internal/affinity/AffinityManager.java
@@ -33,11 +33,11 @@ import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteLogger;
-import org.apache.ignite.metastorage.common.Conditions;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.Operations;
-import org.apache.ignite.metastorage.common.WatchEvent;
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.metastorage.client.Conditions;
+import org.apache.ignite.metastorage.client.EntryEvent;
+import org.apache.ignite.metastorage.client.Operations;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.apache.ignite.network.ClusterNode;
 import org.jetbrains.annotations.NotNull;
 
@@ -85,9 +85,9 @@ public class AffinityManager extends Producer<AffinityEvent, AffinityEventParame
         this.baselineMgr = baselineMgr;
         this.vaultMgr = vaultMgr;
 
-        metaStorageMgr.registerWatchByPrefix(new Key(INTERNAL_PREFIX), new WatchListener() {
-            @Override public boolean onUpdate(@NotNull Iterable<WatchEvent> events) {
-                for (WatchEvent evt : events) {
+        metaStorageMgr.registerWatchByPrefix(new ByteArray(INTERNAL_PREFIX), new WatchListener() {
+            @Override public boolean onUpdate(@NotNull WatchEvent watchEvt) {
+                for (EntryEvent evt : watchEvt.entryEvents()) {
                     String tabIdVal = evt.newEntry().key().toString().substring(INTERNAL_PREFIX.length());
 
                     UUID tblId = UUID.fromString(tabIdVal);
@@ -112,7 +112,7 @@ public class AffinityManager extends Producer<AffinityEvent, AffinityEventParame
             }
 
             @Override public void onError(@NotNull Throwable e) {
-                LOG.error("Metastorage listener issue", e);
+                LOG.error("Meta storage listener issue", e);
             }
         });
     }
@@ -127,17 +127,15 @@ public class AffinityManager extends Producer<AffinityEvent, AffinityEventParame
         return vaultMgr
             .get(ByteArray.fromString(INTERNAL_PREFIX + tblId))
             .thenCompose(entry -> {
-                TableConfiguration tblConfig = configurationMgr
-                    .configurationRegistry()
-                    .getConfiguration(TablesConfiguration.KEY)
-                    .tables()
-                    .get(new String(entry.value(), StandardCharsets.UTF_8));
 
-                var key = new Key(INTERNAL_PREFIX + tblId);
+                            TableConfiguration tblConfig = configurationMgr.configurationRegistry()
+                                    .getConfiguration(TablesConfiguration.KEY).tables().get(new String(entry.value(), StandardCharsets.UTF_8));
+
+                var key = new ByteArray(INTERNAL_PREFIX + tblId);
 
                 // TODO: https://issues.apache.org/jira/browse/IGNITE-14716 Need to support baseline changes.
                 return metaStorageMgr.invoke(
-                    Conditions.key(key).value().eq(null),
+                    Conditions.notExists(key),
                     Operations.put(key, ByteUtils.toBytes(
                         RendezvousAffinityFunction.assignPartitions(
                             baselineMgr.nodes(),
@@ -157,10 +155,10 @@ public class AffinityManager extends Producer<AffinityEvent, AffinityEventParame
      * @return A future which will complete when assignment is removed.
      */
     public CompletableFuture<Boolean> removeAssignment(UUID tblId) {
-        var key = new Key(INTERNAL_PREFIX + tblId);
+        var key = new ByteArray(INTERNAL_PREFIX + tblId);
 
         return metaStorageMgr.invoke(
-            Conditions.key(key).value().ne(null),
+            Conditions.exists(key),
             Operations.remove(key),
             Operations.noop());
     }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/Cursor.java
similarity index 79%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
copy to modules/core/src/main/java/org/apache/ignite/internal/util/Cursor.java
index dcc1c94..e1ddaf2 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/Cursor.java
@@ -15,13 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.util;
+
+import java.util.Iterator;
 
 /**
  * Closeable cursor.
  *
- * @param <T>
+ * @param <T> Type of elements.
  */
-//TODO: Should be replaced by common entity from org.ignite.lang or org.ignite.core package.
-public interface Cursor<T> extends Iterable<T>, AutoCloseable {
+public interface Cursor<T> extends Iterator<T>, Iterable<T>, AutoCloseable {
+
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java b/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
index c38be87..a5552f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
@@ -22,15 +22,17 @@ import java.util.Arrays;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * A class for handling byte array.
+ * A class wraps {@code byte[]} which provides {@link Object#equals}, {@link Object#hashCode} and
+ * lexicographical comparison implementation.
  */
 public final class ByteArray implements Comparable<ByteArray> {
-    /** Byte-wise representation of the {@code ByteArray}. */
+    /** Wrapped byte array. */
     @NotNull
     private final byte[] arr;
 
     /**
-     * Constructs {@code ByteArray} instance from the given byte array. <em>Note:</em> copy of the given byte array will not be
+     * Constructs {@code ByteArray} instance from the given byte array.
+     * <em>Note:</em> copy of the given byte array will not be
      * created in order to avoid redundant memory consumption.
      *
      * @param arr Byte array. Can't be {@code null}.
@@ -40,6 +42,15 @@ public final class ByteArray implements Comparable<ByteArray> {
     }
 
     /**
+     * Constructs {@code ByteArray} instance from the given string.
+     *
+     * @param s The string key representation. Can't be {@code null}.
+     */
+    public ByteArray(@NotNull String s) {
+        this(s.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
      * Constructs {@code ByteArray} instance from the given string. {@link StandardCharsets#UTF_8} charset is used for
      * encoding the input string.
      *
diff --git a/modules/metastorage-client/pom.xml b/modules/metastorage-client/pom.xml
index d45bafd..9cb157f 100644
--- a/modules/metastorage-client/pom.xml
+++ b/modules/metastorage-client/pom.xml
@@ -74,5 +74,11 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-metastorage-server</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
index f929c29..2ffdd30 100644
--- a/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
+++ b/modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ITMetaStorageServiceTest.java
@@ -18,37 +18,40 @@
 package org.apache.ignite.internal.metastorage.client;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
-import org.apache.ignite.internal.metastorage.common.DummyEntry;
-import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageCommandListener;
+import org.apache.ignite.metastorage.client.CompactedException;
+import org.apache.ignite.metastorage.client.Condition;
+import org.apache.ignite.metastorage.client.Conditions;
+import org.apache.ignite.metastorage.client.Entry;
+import org.apache.ignite.metastorage.client.EntryEvent;
 import org.apache.ignite.metastorage.client.MetaStorageService;
-import org.apache.ignite.metastorage.common.CompactedException;
-import org.apache.ignite.metastorage.common.Condition;
-import org.apache.ignite.metastorage.common.Cursor;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.KeyValueStorage;
-import org.apache.ignite.metastorage.common.Operation;
-import org.apache.ignite.metastorage.common.OperationTimeoutException;
-import org.apache.ignite.metastorage.common.WatchEvent;
-import org.apache.ignite.metastorage.common.WatchListener;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.metastorage.client.Operation;
+import org.apache.ignite.metastorage.client.OperationTimeoutException;
+import org.apache.ignite.metastorage.client.Operations;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.apache.ignite.network.ClusterLocalConfiguration;
 import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.ClusterServiceFactory;
 import org.apache.ignite.network.message.MessageSerializationRegistry;
-import org.apache.ignite.network.scalecube.TestScaleCubeClusterServiceFactory;
+import org.apache.ignite.network.scalecube.ScaleCubeClusterServiceFactory;
 import org.apache.ignite.network.scalecube.message.ScaleCubeMessage;
 import org.apache.ignite.network.scalecube.message.ScaleCubeMessageSerializationFactory;
 import org.apache.ignite.raft.client.Peer;
@@ -63,7 +66,6 @@ import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
-
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -95,55 +97,76 @@ public class ITMetaStorageServiceTest {
     public static final int LATEST_REVISION = -1;
 
     /** Factory. */
-    private static RaftClientMessageFactory FACTORY = new RaftClientMessageFactoryImpl();
+    private static final RaftClientMessageFactory FACTORY = new RaftClientMessageFactoryImpl();
 
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final ClusterServiceFactory NETWORK_FACTORY = new ScaleCubeClusterServiceFactory();
 
     /** */
+    // TODO: IGNITE-14088 Uncomment and use real serializer provider
+    //    private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistry();
     private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistry()
-        .registerFactory(ScaleCubeMessage.TYPE, new ScaleCubeMessageSerializationFactory());
-
-    /** Expected result entry. */
-    private static final Entry EXPECTED_RESULT_ENTRY =
-        new DummyEntry(
-            new Key(new byte[] {1}),
-            new byte[] {2},
-            10,
-            2
-        );
+            .registerFactory(ScaleCubeMessage.TYPE, new ScaleCubeMessageSerializationFactory());
+
+    /**  Expected server result entry. */
+    private static final org.apache.ignite.internal.metastorage.server.Entry EXPECTED_SRV_RESULT_ENTRY =
+            new org.apache.ignite.internal.metastorage.server.Entry(
+                    new byte[] {1},
+                    new byte[] {2},
+                    10,
+                    2
+            );
+
+    /**  Expected server result entry. */
+    private static final EntryImpl EXPECTED_RESULT_ENTRY =
+            new EntryImpl(
+                    new ByteArray(new byte[] {1}),
+                    new byte[] {2},
+                    10,
+                    2
+            );
 
     /** Expected result map. */
-    private static final NavigableMap<Key, Entry> EXPECTED_RESULT_MAP;
+    private static final NavigableMap<ByteArray, Entry> EXPECTED_RESULT_MAP;
+
+    private static final Collection<org.apache.ignite.internal.metastorage.server.Entry> EXPECTED_SRV_RESULT_COLL;
 
     /** Cluster. */
     private ArrayList<ClusterService> cluster = new ArrayList<>();
 
-    /** Meta storage raft server. */
+    /**  Meta storage raft server. */
     private RaftServer metaStorageRaftSrv;
 
     static {
         EXPECTED_RESULT_MAP = new TreeMap<>();
 
-        EXPECTED_RESULT_MAP.put(
-            new Key(new byte[] {1}),
-            new DummyEntry(
-                new Key(new byte[] {1}),
-                new byte[] {2},
+        EntryImpl entry1 = new EntryImpl(
+                new ByteArray(new byte[]{1}),
+                new byte[]{2},
                 10,
                 2
-            )
         );
 
-        EXPECTED_RESULT_MAP.put(
-            new Key(new byte[] {3}),
-            new DummyEntry(
-                new Key(new byte[] {3}),
-                new byte[] {4},
+        EXPECTED_RESULT_MAP.put(entry1.key(), entry1);
+
+        EntryImpl entry2 = new EntryImpl(
+                new ByteArray(new byte[]{3}),
+                new byte[]{4},
                 10,
                 3
-            )
         );
+
+        EXPECTED_RESULT_MAP.put(entry2.key(), entry2);
+
+        EXPECTED_SRV_RESULT_COLL = new ArrayList<>();
+
+        EXPECTED_SRV_RESULT_COLL.add(new org.apache.ignite.internal.metastorage.server.Entry(
+                entry1.key().bytes(), entry1.value(), entry1.revision(), entry1.updateCounter()
+        ));
+
+        EXPECTED_SRV_RESULT_COLL.add(new org.apache.ignite.internal.metastorage.server.Entry(
+                entry2.key().bytes(), entry2.value(), entry2.revision(), entry2.updateCounter()
+        ));
     }
 
     /**
@@ -153,11 +176,11 @@ public class ITMetaStorageServiceTest {
     public void beforeTest() {
         for (int i = 0; i < NODES; i++) {
             cluster.add(
-                startClusterNode(
-                    "node_" + i,
-                    NODE_PORT_BASE + i,
-                    IntStream.range(NODE_PORT_BASE, NODE_PORT_BASE + NODES).boxed().
-                        map((port) -> "localhost:" + port).collect(Collectors.toList())));
+                    startClusterNode(
+                            "node_" + i,
+                            NODE_PORT_BASE + i,
+                            IntStream.range(NODE_PORT_BASE, NODE_PORT_BASE + NODES).boxed().
+                                    map((port) -> "localhost:" + port).collect(Collectors.toList())));
         }
 
         for (ClusterService node : cluster)
@@ -180,108 +203,105 @@ public class ITMetaStorageServiceTest {
     }
 
     /**
-     * Tests {@link MetaStorageService#get(Key)}.
+     * Tests {@link MetaStorageService#get(ByteArray)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGet() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull Entry get(byte[] key) {
-                    return EXPECTED_RESULT_ENTRY;
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key) {
+                        return EXPECTED_SRV_RESULT_ENTRY;
+                    }
+                });
 
         assertEquals(EXPECTED_RESULT_ENTRY, metaStorageSvc.get(EXPECTED_RESULT_ENTRY.key()).get());
     }
 
     /**
-     * Tests {@link MetaStorageService#get(Key, long)}.
+     * Tests {@link MetaStorageService#get(ByteArray, long)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGetWithUpperBoundRevision() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull Entry get(byte[] key, long rev) {
-                    return EXPECTED_RESULT_ENTRY;
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key, long rev) {
+                        return EXPECTED_SRV_RESULT_ENTRY;
+                    }
+                });
 
         assertEquals(
-            EXPECTED_RESULT_ENTRY,
-            metaStorageSvc.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision()).get()
+                EXPECTED_RESULT_ENTRY,
+                metaStorageSvc.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision()).get()
         );
     }
 
     /**
-     * Tests {@link MetaStorageService#getAll(Collection)}.
+     * Tests {@link MetaStorageService#getAll(Set)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGetAll() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull List<Entry> getAll(List<byte[]> keys) {
-                    return new ArrayList<>(EXPECTED_RESULT_MAP.values());
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull List<org.apache.ignite.internal.metastorage.server.Entry> getAll(List<byte[]> keys) {
+                        return new ArrayList<>(EXPECTED_SRV_RESULT_COLL);
+                    }
+                });
 
-        assertEquals(
-            EXPECTED_RESULT_MAP,
-            metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet()).get()
-        );
+        assertEquals(EXPECTED_RESULT_MAP, metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet()).get());
     }
 
     /**
-     * Tests {@link MetaStorageService#getAll(Collection, long)}.
+     * Tests {@link MetaStorageService#getAll(Set, long)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGetAllWithUpperBoundRevision() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull List<Entry> getAll(List<byte[]> keys, long revUpperBound) {
-                    return new ArrayList<>(EXPECTED_RESULT_MAP.values());
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull List<org.apache.ignite.internal.metastorage.server.Entry> getAll(List<byte[]> keys, long revUpperBound) {
+                        return new ArrayList<>(EXPECTED_SRV_RESULT_COLL);
+                    }
+                });
 
         assertEquals(
-            EXPECTED_RESULT_MAP,
-            metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
+                EXPECTED_RESULT_MAP,
+                metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
         );
     }
 
     /**
-     * Tests {@link MetaStorageService#put(Key, byte[])}.
+     * Tests {@link MetaStorageService#put(ByteArray, byte[])}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testPut() throws Exception {
-        Key expKey = new Key(new byte[]{1});
+        ByteArray expKey = new ByteArray(new byte[]{1});
 
         byte[] expVal = new byte[]{2};
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @SuppressWarnings("JavaAbbreviationUsage")
-                @Override public void put(byte[] key, byte[] value) {
-                    assertArrayEquals(expKey.bytes(), key);
+                new AbstractKeyValueStorage() {
+                    @SuppressWarnings("JavaAbbreviationUsage")
+                    @Override public void put(byte[] key, byte[] value) {
+                        assertArrayEquals(expKey.bytes(), key);
 
-                    assertArrayEquals(expVal, value);
-                }
-            });
+                        assertArrayEquals(expVal, value);
+                    }
+                });
 
         metaStorageSvc.put(expKey, expVal).get();
     }
 
     /**
-     * Tests {@link MetaStorageService#getAndPut(Key, byte[])}.
+     * Tests {@link MetaStorageService#getAndPut(ByteArray, byte[])}.
      *
      * @throws Exception If failed.
      */
@@ -290,20 +310,20 @@ public class ITMetaStorageServiceTest {
         byte[] expVal = new byte[]{2};
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @SuppressWarnings("JavaAbbreviationUsage")
-                @Override public @NotNull Entry getAndPut(byte[] key, byte[] value) {
-                    assertArrayEquals(EXPECTED_RESULT_ENTRY.key().bytes(), key);
+                new AbstractKeyValueStorage() {
+                    @SuppressWarnings("JavaAbbreviationUsage")
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry getAndPut(byte[] key, byte[] value) {
+                        assertArrayEquals(EXPECTED_RESULT_ENTRY.key().bytes(), key);
 
-                    assertArrayEquals(expVal, value);
+                        assertArrayEquals(expVal, value);
 
-                    return EXPECTED_RESULT_ENTRY;
-                }
-            });
+                        return EXPECTED_SRV_RESULT_ENTRY;
+                    }
+                });
 
         assertEquals(
-            EXPECTED_RESULT_ENTRY,
-            metaStorageSvc.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal).get()
+                EXPECTED_RESULT_ENTRY,
+                metaStorageSvc.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal).get()
         );
     }
 
@@ -315,35 +335,35 @@ public class ITMetaStorageServiceTest {
     @Test
     public void testPutAll() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public void putAll(List<byte[]> keys, List<byte[]> values) {
-                    // Assert keys equality.
-                    assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
+                new AbstractKeyValueStorage() {
+                    @Override public void putAll(List<byte[]> keys, List<byte[]> values) {
+                        // Assert keys equality.
+                        assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
 
-                    List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
-                        map(Key::bytes).collect(Collectors.toList());
+                        List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
+                                map(ByteArray::bytes).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expKeys.get(i), keys.get(i));
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expKeys.get(i), keys.get(i));
 
 
-                    // Assert values equality.
-                    assertEquals(EXPECTED_RESULT_MAP.values().size(), values.size());
+                        // Assert values equality.
+                        assertEquals(EXPECTED_RESULT_MAP.values().size(), values.size());
 
-                    List<byte[]> expVals = EXPECTED_RESULT_MAP.values().stream().
-                        map(Entry::value).collect(Collectors.toList());
+                        List<byte[]> expVals = EXPECTED_RESULT_MAP.values().stream().
+                                map(Entry::value).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expVals.get(i), values.get(i));
-                }
-            });
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expVals.get(i), values.get(i));
+                    }
+                });
 
         metaStorageSvc.putAll(
-            EXPECTED_RESULT_MAP.entrySet().stream()
-                .collect(Collectors.toMap(
-                    Map.Entry::getKey,
-                    e -> e.getValue().value())
-                )
+                EXPECTED_RESULT_MAP.entrySet().stream()
+                        .collect(Collectors.toMap(
+                                Map.Entry::getKey,
+                                e -> e.getValue().value())
+                        )
         ).get();
     }
 
@@ -355,297 +375,349 @@ public class ITMetaStorageServiceTest {
     @Test
     public void testGetAndPutAll() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull List<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
-                    // Assert keys equality.
-                    assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull List<org.apache.ignite.internal.metastorage.server.Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
+                        // Assert keys equality.
+                        assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
 
-                    List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
-                        map(Key::bytes).collect(Collectors.toList());
+                        List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
+                                map(ByteArray::bytes).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expKeys.get(i), keys.get(i));
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expKeys.get(i), keys.get(i));
 
-                    // Assert values equality.
-                    assertEquals(EXPECTED_RESULT_MAP.values().size(), values.size());
+                        // Assert values equality.
+                        assertEquals(EXPECTED_RESULT_MAP.values().size(), values.size());
 
-                    List<byte[]> expVals = EXPECTED_RESULT_MAP.values().stream().
-                        map(Entry::value).collect(Collectors.toList());
+                        List<byte[]> expVals = EXPECTED_RESULT_MAP.values().stream().
+                                map(Entry::value).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expVals.get(i), values.get(i));
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expVals.get(i), values.get(i));
 
-                    return new ArrayList<>(EXPECTED_RESULT_MAP.values());
-                }
-            });
+                        return new ArrayList<>(EXPECTED_SRV_RESULT_COLL);
+                    }
+                });
 
-        Map<Key, Entry> gotRes = metaStorageSvc.getAndPutAll(
-            EXPECTED_RESULT_MAP.entrySet().stream()
-                .collect(Collectors.toMap(
-                    Map.Entry::getKey,
-                    e -> e.getValue().value())
-                )
+        Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndPutAll(
+                EXPECTED_RESULT_MAP.entrySet().stream()
+                        .collect(Collectors.toMap(
+                                Map.Entry::getKey,
+                                e -> e.getValue().value())
+                        )
         ).get();
 
         assertEquals(EXPECTED_RESULT_MAP, gotRes);
     }
 
     /**
-     * Tests {@link MetaStorageService#remove(Key)}.
+     * Tests {@link MetaStorageService#remove(ByteArray)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRemove() throws Exception {
-        Key expKey = new Key(new byte[]{1});
+        ByteArray expKey = new ByteArray(new byte[]{1});
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public void remove(byte[] key) {
-                    assertArrayEquals(expKey.bytes(), key);
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public void remove(byte[] key) {
+                        assertArrayEquals(expKey.bytes(), key);
+                    }
+                });
 
         metaStorageSvc.remove(expKey).get();
     }
 
     /**
-     * Tests {@link MetaStorageService#getAndRemove(Key)}.
+     * Tests {@link MetaStorageService#getAndRemove(ByteArray)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGetAndRemove() throws Exception {
-        Entry expRes = new DummyEntry(
-            new Key(new byte[]{1}),
-            new byte[]{3},
-            10,
-            2
+        EntryImpl expRes = new EntryImpl(
+                new ByteArray(new byte[]{1}),
+                new byte[]{3},
+                10,
+                2
         );
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull Entry getAndRemove(byte[] key) {
-                    assertArrayEquals(expRes.key().bytes(), key);
-
-                    return expRes;
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry getAndRemove(byte[] key) {
+                        assertArrayEquals(expRes.key().bytes(), key);
+
+                        return new org.apache.ignite.internal.metastorage.server.Entry(
+                                expRes.key().bytes(),
+                                expRes.value(),
+                                expRes.revision(),
+                                expRes.updateCounter()
+                        );
+                    }
+                });
 
         assertEquals(expRes, metaStorageSvc.getAndRemove(expRes.key()).get());
     }
 
     /**
-     * Tests {@link MetaStorageService#removeAll(Collection)}.
+     * Tests {@link MetaStorageService#removeAll(Set)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRemoveAll() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public void removeAll(List<byte[]> keys) {
-                    assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
+                new AbstractKeyValueStorage() {
+                    @Override public void removeAll(List<byte[]> keys) {
+                        assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
 
-                    List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
-                        map(Key::bytes).collect(Collectors.toList());
+                        List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
+                                map(ByteArray::bytes).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expKeys.get(i), keys.get(i));
-                }
-            });
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expKeys.get(i), keys.get(i));
+                    }
+                });
 
         metaStorageSvc.removeAll(EXPECTED_RESULT_MAP.keySet()).get();
     }
 
     /**
-     * Tests {@link MetaStorageService#getAndRemoveAll(Collection)}.
+     * Tests {@link MetaStorageService#getAndRemoveAll(Set)}.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testGetAndRemoveAll() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull List<Entry> getAndRemoveAll(List<byte[]> keys) {
-                    // Assert keys equality.
-                    assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull List<org.apache.ignite.internal.metastorage.server.Entry> getAndRemoveAll(List<byte[]> keys) {
+                        // Assert keys equality.
+                        assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keys.size());
 
-                    List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
-                        map(Key::bytes).collect(Collectors.toList());
+                        List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream().
+                                map(ByteArray::bytes).collect(Collectors.toList());
 
-                    for (int i = 0; i < expKeys.size(); i++)
-                        assertArrayEquals(expKeys.get(i), keys.get(i));
+                        for (int i = 0; i < expKeys.size(); i++)
+                            assertArrayEquals(expKeys.get(i), keys.get(i));
 
-                    return new ArrayList<>(EXPECTED_RESULT_MAP.values());
-                }
-            });
+                        return new ArrayList<>(EXPECTED_SRV_RESULT_COLL);
+                    }
+                });
 
-        Map<Key, Entry> gotRes = metaStorageSvc.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
+        Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
 
         assertEquals(EXPECTED_RESULT_MAP, gotRes);
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} with not null keyTo and explicit revUpperBound.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} with not null keyTo and explicit revUpperBound.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeWitKeyToAndUpperBound() throws Exception {
-        Key expKeyFrom = new Key(new byte[]{1});
+        ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
-        Key expKeyTo = new Key(new byte[]{3});
+        ByteArray expKeyTo = new ByteArray(new byte[]{3});
 
         long expRevUpperBound = 10;
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    assertArrayEquals(expKeyFrom.bytes(), keyFrom);
-
-                    assertArrayEquals(expKeyTo.bytes(), keyTo);
-
-                    assertEquals(expRevUpperBound, revUpperBound);
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        assertArrayEquals(expKeyFrom.bytes(), keyFrom);
 
-                    return new Cursor<>() {
-                        @Override public void close() throws Exception {
+                        assertArrayEquals(expKeyTo.bytes(), keyTo);
 
-                        }
+                        assertEquals(expRevUpperBound, revUpperBound);
 
-                        @NotNull @Override public Iterator<Entry> iterator() {
-                            return new Iterator<>() {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.Entry> it = new Iterator<>() {
                                 @Override public boolean hasNext() {
                                     return false;
                                 }
 
-                                @Override public Entry next() {
+                                @Override public org.apache.ignite.internal.metastorage.server.Entry next() {
                                     return null;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+
+                            @Override public void close() throws Exception {
+
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.Entry> iterator() {
+                                return it;
+                            }
+
+                            @Override
+                            public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override
+                            public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         metaStorageSvc.range(expKeyFrom, expKeyTo, expRevUpperBound).close();
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} with not null keyTo.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} with not null keyTo.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeWitKeyTo() throws Exception {
-        Key expKeyFrom = new Key(new byte[]{1});
+        ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
-        Key expKeyTo = new Key(new byte[]{3});
+        ByteArray expKeyTo = new ByteArray(new byte[]{3});
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    assertArrayEquals(expKeyFrom.bytes(), keyFrom);
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        assertArrayEquals(expKeyFrom.bytes(), keyFrom);
 
-                    assertArrayEquals(expKeyTo.bytes(), keyTo);
+                        assertArrayEquals(expKeyTo.bytes(), keyTo);
 
-                    assertEquals(LATEST_REVISION, revUpperBound);
+                        assertEquals(LATEST_REVISION, revUpperBound);
 
-                    return new Cursor<>() {
-                        @Override public void close() throws Exception {
-
-                        }
-
-                        @NotNull @Override public Iterator<Entry> iterator() {
-                            return new Iterator<>() {
-                                @Override public boolean hasNext() {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.Entry> it = new Iterator<>() {
+                                @Override
+                                public boolean hasNext() {
                                     return false;
                                 }
 
-                                @Override public Entry next() {
+                                @Override
+                                public org.apache.ignite.internal.metastorage.server.Entry next() {
                                     return null;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+                            @Override public void close() throws Exception {
+
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.Entry> iterator() {
+                                return it;
+                            }
+
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         metaStorageSvc.range(expKeyFrom, expKeyTo).close();
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} with null keyTo.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} with null keyTo.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeWitNullAsKeyTo() throws Exception {
-        Key expKeyFrom = new Key(new byte[]{1});
+        ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    assertArrayEquals(expKeyFrom.bytes(), keyFrom);
-
-                    assertNull(keyTo);
-
-                    assertEquals(LATEST_REVISION, revUpperBound);
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        assertArrayEquals(expKeyFrom.bytes(), keyFrom);
 
-                    return new Cursor<>() {
-                        @Override public void close() throws Exception {
+                        assertNull(keyTo);
 
-                        }
+                        assertEquals(LATEST_REVISION, revUpperBound);
 
-                        @NotNull @Override public Iterator<Entry> iterator() {
-                            return new Iterator<>() {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.Entry> it = new Iterator<>() {
                                 @Override public boolean hasNext() {
                                     return false;
                                 }
 
-                                @Override public Entry next() {
+                                @Override public org.apache.ignite.internal.metastorage.server.Entry next() {
                                     return null;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+                            @Override public void close() throws Exception {
+
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.Entry> iterator() {
+                                return it;
+                            }
+
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override
+                            public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         metaStorageSvc.range(expKeyFrom, null).close();
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} hasNext.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} hasNext.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeHasNext() throws Exception {
-        Key expKeyFrom = new Key(new byte[]{1});
+        ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    return new Cursor<>() {
-                        @Override public void close() throws Exception {
-
-                        }
-
-                        @NotNull @Override public Iterator<Entry> iterator() {
-                            return new Iterator<>() {
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.Entry> it = new Iterator<>() {
                                 @Override public boolean hasNext() {
                                     return true;
                                 }
 
-                                @Override public Entry next() {
+                                @Override public org.apache.ignite.internal.metastorage.server.Entry next() {
                                     return null;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+                            @Override public void close() throws Exception {
+
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.Entry> iterator() {
+                                return it;
+                            }
+
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override
+                            public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
 
@@ -653,34 +725,45 @@ public class ITMetaStorageServiceTest {
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} next.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} next.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeNext() throws Exception {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    return new Cursor<>() {
-                        @Override public void close() throws Exception {
-
-                        }
-
-                        @NotNull @Override public Iterator<Entry> iterator() {
-                            return new Iterator<>() {
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.Entry> it = new Iterator<>() {
                                 @Override public boolean hasNext() {
                                     return true;
                                 }
 
-                                @Override public Entry next() {
-                                    return EXPECTED_RESULT_ENTRY;
+                                @Override public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                    return EXPECTED_SRV_RESULT_ENTRY;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+                            @Override public void close() throws Exception {
+
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.Entry> iterator() {
+                                return it;
+                            }
+
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override
+                            public org.apache.ignite.internal.metastorage.server.Entry next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         Cursor<Entry> cursor = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
@@ -688,22 +771,22 @@ public class ITMetaStorageServiceTest {
     }
 
     /**
-     * Tests {@link MetaStorageService#range(Key, Key, long)}} close.
+     * Tests {@link MetaStorageService#range(ByteArray, ByteArray, long)}} close.
      *
      * @throws Exception If failed.
      */
     @Test
     public void testRangeClose() throws Exception {
-        Key expKeyFrom = new Key(new byte[]{1});
+        ByteArray expKeyFrom = new ByteArray(new byte[]{1});
 
         Cursor cursorMock = mock(Cursor.class);
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-                    return cursorMock;
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+                        return cursorMock;
+                    }
+                });
 
         Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
 
@@ -714,101 +797,115 @@ public class ITMetaStorageServiceTest {
 
     @Test
     public void testWatchOnUpdate() throws Exception {
-        List<WatchEvent> returnedWatchEvents = Arrays.asList(
-            new WatchEvent(
-                new DummyEntry(
-                    new Key(new byte[]{2}),
-                    new byte[]{20},
-                    1,
-                    1
-                ),
-                new DummyEntry(
-                    new Key(new byte[]{2}),
-                    new byte[]{21},
-                    2,
-                    4
-                )
-            ),
-            new WatchEvent(
-                new DummyEntry(
-                    new Key(new byte[] {3}),
-                    new byte[] {20},
-                    1,
-                    2
+        org.apache.ignite.internal.metastorage.server.WatchEvent returnedWatchEvents = new org.apache.ignite.internal.metastorage.server.WatchEvent(List.of(
+                new org.apache.ignite.internal.metastorage.server.EntryEvent(
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[]{2},
+                                new byte[]{20},
+                                1,
+                                1
+                        ),
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[]{2},
+                                new byte[]{21},
+                                2,
+                                4
+                        )
                 ),
-                new DummyEntry(
-                    new Key(new byte[] {3}),
-                    null,
-                    2,
-                    5
-                )
-            ),
-            new WatchEvent(
-                new DummyEntry(
-                    new Key(new byte[] {4}),
-                    new byte[] {20},
-                    1,
-                    3
+                new org.apache.ignite.internal.metastorage.server.EntryEvent(
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[] {3},
+                                new byte[] {20},
+                                1,
+                                2
+                        ),
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[] {3},
+                                new byte[]{},
+                                2,
+                                5
+                        )
                 ),
-                new DummyEntry(
-                    new Key(new byte[] {4}),
-                    null,
-                    3,
-                    6
+                new org.apache.ignite.internal.metastorage.server.EntryEvent(
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[] {4},
+                                new byte[] {20},
+                                1,
+                                3
+                        ),
+                        new org.apache.ignite.internal.metastorage.server.Entry(
+                                new byte[] {4},
+                                new byte[] {},
+                                3,
+                                6
+                        )
                 )
-            )
-        );
+        ));
 
-        Key keyFrom = new Key(new byte[]{1});
+        ByteArray keyFrom = new ByteArray(new byte[]{1});
 
-        Key keyTo = new Key(new byte[]{10});
+        ByteArray keyTo = new ByteArray(new byte[]{10});
 
         long rev = 2;
 
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
-                    return new Cursor<>() {
-                        AtomicInteger retirevedItemCnt = new AtomicInteger(0);
-
-                        @Override public void close() throws Exception {
-                            // No-op.
-                        }
-
-                        @NotNull @Override public Iterator<WatchEvent> iterator() {
-                            return new Iterator<WatchEvent>() {
+                new AbstractKeyValueStorage() {
+                    @Override public Cursor<org.apache.ignite.internal.metastorage.server.WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
+                        return new Cursor<>() {
+                            private final Iterator<org.apache.ignite.internal.metastorage.server.WatchEvent> it = new Iterator<>() {
                                 @Override public boolean hasNext() {
 
-                                    return retirevedItemCnt.get() < returnedWatchEvents.size();
+                                    return retirevedItemCnt.get() < returnedWatchEvents.entryEvents().size();
                                 }
 
-                                @Override public WatchEvent next() {
-                                    return returnedWatchEvents.get(retirevedItemCnt.getAndIncrement());
+                                @Override public org.apache.ignite.internal.metastorage.server.WatchEvent next() {
+                                    return returnedWatchEvents;
                                 }
                             };
-                        }
-                    };
-                }
-            });
+
+                            AtomicInteger retirevedItemCnt = new AtomicInteger(0);
+
+                            @Override public void close() throws Exception {
+                                // No-op.
+                            }
+
+                            @NotNull @Override public Iterator<org.apache.ignite.internal.metastorage.server.WatchEvent> iterator() {
+                                return it;
+                            }
+
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
+
+                            @Override
+                            public org.apache.ignite.internal.metastorage.server.WatchEvent next() {
+                                return it.next();
+                            }
+                        };
+                    }
+                });
 
         CountDownLatch latch = new CountDownLatch(1);
 
         IgniteUuid watchId = metaStorageSvc.watch(keyFrom, keyTo, rev, new WatchListener() {
-            @Override public boolean onUpdate(@NotNull Iterable<WatchEvent> events) {
+            @Override public boolean onUpdate(@NotNull WatchEvent events) {
                 List gotEvents = new ArrayList();
 
-                Iterator<WatchEvent> iter = events.iterator();
+                List returnedWatchEvents = new ArrayList(events.entryEvents());
+
+                Iterator<EntryEvent> iter = events.entryEvents().iterator();
 
                 while (iter.hasNext())
                     gotEvents.add(iter.next());
 
-                assertEquals(2, gotEvents.size());
+                assertEquals(3, gotEvents.size());
 
                 assertTrue(gotEvents.contains(returnedWatchEvents.get(0)));
 
                 assertTrue(gotEvents.contains(returnedWatchEvents.get(1)));
 
                 latch.countDown();
+
                 return true;
             }
 
@@ -823,11 +920,43 @@ public class ITMetaStorageServiceTest {
         metaStorageSvc.stopWatch(watchId).get();
     }
 
+    @Test
+    public void testInvoke() throws Exception {
+        ByteArray expKey = new ByteArray(new byte[]{1});
+
+        byte[] expVal = new byte[]{2};
+
+        Condition condition = Conditions.notExists(expKey);
+
+        Operation success = Operations.put(expKey, expVal);
+
+        Operation failure = Operations.noop();
+
+        MetaStorageService metaStorageSvc = prepareMetaStorage(
+                new AbstractKeyValueStorage() {
+                    @Override public boolean invoke(
+                            org.apache.ignite.internal.metastorage.server.Condition cond,
+                            Collection<org.apache.ignite.internal.metastorage.server.Operation> success,
+                            Collection<org.apache.ignite.internal.metastorage.server.Operation> failure) {
+                        assertArrayEquals(expKey.bytes(), cond.key());
+
+                        assertArrayEquals(expKey.bytes(), success.iterator().next().key());
+                        assertArrayEquals(expVal, success.iterator().next().value());
+
+                        assertEquals(OperationType.NO_OP, failure.iterator().next().type());
+
+                        return true;
+                    }
+                });
+
+        assertTrue(metaStorageSvc.invoke(condition, success, failure).get());
+    }
+
     // TODO: IGNITE-14693 Add tests for exception handling logic: onError,
     // TODO: (CompactedException | OperationTimeoutException)
 
     /**
-     * Tests {@link MetaStorageService#get(Key)}.
+     * Tests {@link MetaStorageService#get(ByteArray)}.
      *
      * @throws Exception If failed.
      */
@@ -835,17 +964,17 @@ public class ITMetaStorageServiceTest {
     @Test
     public void testGetThatThrowsCompactedException() {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull Entry get(byte[] key) {
-                    throw new CompactedException();
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key) {
+                        throw new org.apache.ignite.internal.metastorage.server.CompactedException();
+                    }
+                });
 
         assertThrows(CompactedException.class, () -> metaStorageSvc.get(EXPECTED_RESULT_ENTRY.key()).get());
     }
 
     /**
-     * Tests {@link MetaStorageService#get(Key)}.
+     * Tests {@link MetaStorageService#get(ByteArray)}.
      *
      * @throws Exception If failed.
      */
@@ -853,11 +982,11 @@ public class ITMetaStorageServiceTest {
     @Test
     public void testGetThatThrowsOperationTimeoutException() {
         MetaStorageService metaStorageSvc = prepareMetaStorage(
-            new AbstractKeyValueStorage() {
-                @Override public @NotNull Entry get(byte[] key) {
-                    throw new OperationTimeoutException();
-                }
-            });
+                new AbstractKeyValueStorage() {
+                    @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key) {
+                        throw new OperationTimeoutException();
+                    }
+                });
 
         assertThrows(OperationTimeoutException.class, () -> metaStorageSvc.get(EXPECTED_RESULT_ENTRY.key()).get());
     }
@@ -904,7 +1033,7 @@ public class ITMetaStorageServiceTest {
     }
 
     /**
-     * Prepares meta storage by instantiating corresponding raft server with {@link MetaStorageCommandListener} and
+     * Prepares meta storage by instantiating corresponding raft server with MetaStorageCommandListener and
      * {@link MetaStorageServiceImpl}.
      *
      * @param keyValStorageMock {@link KeyValueStorage} mock.
@@ -912,20 +1041,20 @@ public class ITMetaStorageServiceTest {
      */
     private MetaStorageService prepareMetaStorage(KeyValueStorage keyValStorageMock) {
         metaStorageRaftSrv = new RaftServerImpl(
-            cluster.get(0),
-            FACTORY,
-            1000,
-            Map.of(METASTORAGE_RAFT_GROUP_NAME, new MetaStorageCommandListener(keyValStorageMock))
+                cluster.get(0),
+                FACTORY,
+                1000,
+                Map.of(METASTORAGE_RAFT_GROUP_NAME, new MetaStorageCommandListener(keyValStorageMock))
         );
 
         RaftGroupService metaStorageRaftGrpSvc = new RaftGroupServiceImpl(
-            METASTORAGE_RAFT_GROUP_NAME,
-            cluster.get(1),
-            FACTORY,
-            10_000,
-            List.of(new Peer(cluster.get(0).topologyService().localMember())),
-            true,
-            200
+                METASTORAGE_RAFT_GROUP_NAME,
+                cluster.get(1),
+                FACTORY,
+                10_000,
+                List.of(new Peer(cluster.get(0).topologyService().localMember())),
+                true,
+                200
         );
 
         return new MetaStorageServiceImpl(metaStorageRaftGrpSvc);
@@ -947,28 +1076,28 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Entry get(byte[] key) {
+        @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Entry get(byte[] key, long rev) {
+        @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry get(byte[] key, long rev) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Collection<Entry> getAll(List<byte[]> keys) {
+        @Override public @NotNull Collection<org.apache.ignite.internal.metastorage.server.Entry> getAll(List<byte[]> keys) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Collection<Entry> getAll(List<byte[]> keys, long revUpperBound) {
+        @Override public @NotNull Collection<org.apache.ignite.internal.metastorage.server.Entry> getAll(List<byte[]> keys, long revUpperBound) {
             fail();
 
             return null;
@@ -980,7 +1109,7 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Entry getAndPut(byte[] key, byte[] value) {
+        @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry getAndPut(byte[] key, byte[] value) {
             fail();
 
             return null;
@@ -992,7 +1121,7 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
+        @Override public @NotNull Collection<org.apache.ignite.internal.metastorage.server.Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
             fail();
 
             return null;
@@ -1004,7 +1133,7 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Entry getAndRemove(byte[] key) {
+        @Override public @NotNull org.apache.ignite.internal.metastorage.server.Entry getAndRemove(byte[] key) {
             fail();
 
             return null;
@@ -1016,7 +1145,7 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public @NotNull Collection<Entry> getAndRemoveAll(List<byte[]> keys) {
+        @Override public @NotNull Collection<org.apache.ignite.internal.metastorage.server.Entry> getAndRemoveAll(List<byte[]> keys) {
             fail();
 
             return null;
@@ -1024,9 +1153,9 @@ public class ITMetaStorageServiceTest {
 
         /** {@inheritDoc} */
         @Override public boolean invoke(
-            Condition condition,
-            Collection<Operation> success,
-            Collection<Operation> failure
+                org.apache.ignite.internal.metastorage.server.Condition condition,
+                Collection<org.apache.ignite.internal.metastorage.server.Operation> success,
+                Collection<org.apache.ignite.internal.metastorage.server.Operation> failure
         ) {
             fail();
 
@@ -1034,35 +1163,35 @@ public class ITMetaStorageServiceTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo) {
+        @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+        @Override public Cursor<org.apache.ignite.internal.metastorage.server.Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
+        @Override public Cursor<org.apache.ignite.internal.metastorage.server.WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public Cursor<WatchEvent> watch(byte[] key, long rev) {
+        @Override public Cursor<org.apache.ignite.internal.metastorage.server.WatchEvent> watch(byte[] key, long rev) {
             fail();
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+        @Override public Cursor<org.apache.ignite.internal.metastorage.server.WatchEvent> watch(Collection<byte[]> keys, long rev) {
             fail();
 
             return null;
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
index 9974266..dbbb2aa 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
@@ -20,13 +20,14 @@ package org.apache.ignite.internal.metastorage.client;
 import java.util.Iterator;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorCloseCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorHasNextCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorNextCommand;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.Cursor;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.jetbrains.annotations.NotNull;
 
@@ -39,49 +40,31 @@ public class CursorImpl<T> implements Cursor<T> {
     private static final IgniteLogger LOG = IgniteLogger.forClass(CursorImpl.class);
 
     /** Future that runs meta storage service operation that provides cursor. */
-    private CompletableFuture<IgniteUuid> initOp;
+    private final CompletableFuture<IgniteUuid> initOp;
 
     /** Meta storage raft group service. */
-    private RaftGroupService metaStorageRaftGrpSvc;
+    private final RaftGroupService metaStorageRaftGrpSvc;
+
+    /** */
+    private final Iterator<T> it;
+
+    /** */
+    private final Function<Object, T> fn;
 
     /**
      * @param metaStorageRaftGrpSvc Meta storage raft group service.
      * @param initOp Future that runs meta storage service operation that provides cursor.
      */
-    CursorImpl(RaftGroupService metaStorageRaftGrpSvc, CompletableFuture<IgniteUuid> initOp) {
+    CursorImpl(RaftGroupService metaStorageRaftGrpSvc, CompletableFuture<IgniteUuid> initOp, Function<Object, T> fn) {
         this.metaStorageRaftGrpSvc = metaStorageRaftGrpSvc;
         this.initOp = initOp;
+        this.it = new InnerIterator();
+        this.fn = fn;
     }
 
     /** {@inheritDoc} */
     @NotNull @Override public Iterator<T> iterator() {
-        return new Iterator<>() {
-            /** {@inheritDoc} */
-            @Override public boolean hasNext() {
-                try {
-                    return initOp.thenCompose(
-                        cursorId -> metaStorageRaftGrpSvc.<Boolean>run(new CursorHasNextCommand(cursorId))).get();
-                }
-                catch (InterruptedException | ExecutionException e) {
-                    LOG.error("Unable to evaluate cursor hasNext command", e);
-
-                    throw new IgniteInternalException(e);
-                }
-            }
-
-            /** {@inheritDoc} */
-            @Override public T next() {
-                try {
-                    return initOp.thenCompose(
-                        cursorId -> metaStorageRaftGrpSvc.<T>run(new CursorNextCommand(cursorId))).get();
-                }
-                catch (InterruptedException | ExecutionException e) {
-                    LOG.error("Unable to evaluate cursor hasNext command", e);
-
-                    throw new IgniteInternalException(e);
-                }
-            }
-        };
+        return it;
     }
 
     /** {@inheritDoc} */
@@ -96,4 +79,43 @@ public class CursorImpl<T> implements Cursor<T> {
             throw new IgniteInternalException(e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() {
+        return it.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public T next() {
+        return it.next();
+    }
+
+    /** */
+    private class InnerIterator implements Iterator<T> {
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            try {
+                return initOp.thenCompose(
+                        cursorId -> metaStorageRaftGrpSvc.<Boolean>run(new CursorHasNextCommand(cursorId))).get();
+            }
+            catch (InterruptedException | ExecutionException e) {
+                LOG.error("Unable to evaluate cursor hasNext command", e);
+
+                throw new IgniteInternalException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public T next() {
+            try {
+                return initOp.thenCompose(
+                        cursorId -> metaStorageRaftGrpSvc.run(new CursorNextCommand(cursorId))).thenApply(fn).get();
+            }
+            catch (InterruptedException | ExecutionException e) {
+                LOG.error("Unable to evaluate cursor hasNext command", e);
+
+                throw new IgniteInternalException(e);
+            }
+        }
+    }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/EntryImpl.java
similarity index 60%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/EntryImpl.java
index c4e6ed8..94164a0 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/EntryImpl.java
@@ -15,90 +15,111 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.metastorage.common;
+package org.apache.ignite.internal.metastorage.client;
 
-import java.io.Serializable;
 import java.util.Arrays;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.metastorage.client.Entry;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
-// TODO: IGNITE-14389 Tmp, used instead of real Entry implementation. Should be removed.
 /**
- * Dummy entry implementation.
+ * Meta storage entry.
  */
-public final class DummyEntry implements Entry, Serializable {
+public final class EntryImpl implements Entry {
     /** Key. */
-    @NotNull private Key key;
+    @NotNull
+    private final ByteArray key;
 
     /** Value. */
-    @Nullable private byte[] val;
+    @Nullable
+    private final byte[] val;
 
     /** Revision. */
-    private long revision;
+    private final long rev;
 
     /** Update counter. */
-    private long updateCntr;
+    private final long updCntr;
 
     /**
+     * Construct entry with given paramteters.
      *
      * @param key Key.
      * @param val Value.
-     * @param revision Revision.
-     * @param updateCntr Update counter.
+     * @param rev Revision.
+     * @param updCntr Update counter.
      */
-    public DummyEntry(@NotNull Key key, @Nullable byte[] val, long revision, long updateCntr) {
+    EntryImpl(@NotNull ByteArray key, @Nullable byte[] val, long rev, long updCntr) {
         this.key = key;
         this.val = val;
-        this.revision = revision;
-        this.updateCntr = updateCntr;
+        this.rev = rev;
+        this.updCntr = updCntr;
     }
 
-    /** {@inheritDoc} */
-    @Override public @NotNull Key key() {
+    /** {@inheritDoc} */    @NotNull
+    @Override public ByteArray key() {
         return key;
     }
 
     /** {@inheritDoc} */
-    @Override public @Nullable byte[] value() {
+    @Nullable
+    @Override public byte[] value() {
         return val;
     }
 
     /** {@inheritDoc} */
     @Override public long revision() {
-        return revision;
+        return rev;
     }
 
     /** {@inheritDoc} */
     @Override public long updateCounter() {
-        return updateCntr;
+        return updCntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean tombstone() {
+        return val == null && rev > 0 && updCntr > 0;
     }
 
     /** {@inheritDoc} */
+    @Override public boolean empty() {
+        return val == null && rev == 0 && updCntr == 0;
+    }
+
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;
+
         if (o == null || getClass() != o.getClass())
             return false;
 
-        DummyEntry entry = (DummyEntry)o;
+        EntryImpl entry = (EntryImpl)o;
 
-        if (revision != entry.revision)
+        if (rev != entry.rev)
             return false;
-        if (updateCntr != entry.updateCntr)
+
+        if (updCntr != entry.updCntr)
             return false;
+
         if (!key.equals(entry.key))
             return false;
+
         return Arrays.equals(val, entry.val);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
         int res = key.hashCode();
+
         res = 31 * res + Arrays.hashCode(val);
-        res = 31 * res + (int)(revision ^ (revision >>> 32));
-        res = 31 * res + (int)(updateCntr ^ (updateCntr >>> 32));
+
+        res = 31 * res + (int)(rev ^ (rev >>> 32));
+
+        res = 31 * res + (int)(updCntr ^ (updCntr >>> 32));
+
         return res;
     }
 }
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
index 4102805..3d7bde9 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
@@ -19,37 +19,44 @@ package org.apache.ignite.internal.metastorage.client;
 
 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.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Function;
-import java.util.stream.Collectors;
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.apache.ignite.internal.metastorage.common.command.ConditionInfo;
 import org.apache.ignite.internal.metastorage.common.command.GetAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndPutAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndPutCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndRemoveAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndRemoveCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetCommand;
+import org.apache.ignite.internal.metastorage.common.command.InvokeCommand;
+import org.apache.ignite.internal.metastorage.common.command.MultipleEntryResponse;
+import org.apache.ignite.internal.metastorage.common.command.OperationInfo;
 import org.apache.ignite.internal.metastorage.common.command.PutAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.PutCommand;
 import org.apache.ignite.internal.metastorage.common.command.RangeCommand;
 import org.apache.ignite.internal.metastorage.common.command.RemoveAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.RemoveCommand;
+import org.apache.ignite.internal.metastorage.common.command.SingleEntryResponse;
 import org.apache.ignite.internal.metastorage.common.command.WatchExactKeysCommand;
 import org.apache.ignite.internal.metastorage.common.command.WatchRangeKeysCommand;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.metastorage.client.Condition;
+import org.apache.ignite.metastorage.client.Entry;
+import org.apache.ignite.metastorage.client.EntryEvent;
 import org.apache.ignite.metastorage.client.MetaStorageService;
-import org.apache.ignite.metastorage.common.Condition;
-import org.apache.ignite.metastorage.common.Cursor;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.Operation;
-import org.apache.ignite.metastorage.common.WatchEvent;
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.metastorage.client.Operation;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -77,111 +84,117 @@ public class MetaStorageServiceImpl implements MetaStorageService {
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> get(@NotNull Key key) {
-        return metaStorageRaftGrpSvc.run(new GetCommand(key));
+    @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key) {
+        return metaStorageRaftGrpSvc.run(new GetCommand(key)).thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> get(@NotNull Key key, long revUpperBound) {
-        return metaStorageRaftGrpSvc.run(new GetCommand(key, revUpperBound));
+    @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound) {
+        return metaStorageRaftGrpSvc.run(new GetCommand(key, revUpperBound))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys) {
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Set<ByteArray> keys) {
+        return metaStorageRaftGrpSvc.run(new GetAllCommand(keys))
+                .thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys, long revUpperBound) {
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys, revUpperBound)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Set<ByteArray> keys, long revUpperBound) {
+        return metaStorageRaftGrpSvc.run(new GetAllCommand(keys, revUpperBound)).
+                thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> put(@NotNull Key key, @NotNull byte[] value) {
+    @Override public @NotNull CompletableFuture<Void> put(@NotNull ByteArray key, @NotNull byte[] value) {
         return metaStorageRaftGrpSvc.run(new PutCommand(key, value));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> getAndPut(@NotNull Key key, @NotNull byte[] value) {
-        return metaStorageRaftGrpSvc.run(new GetAndPutCommand(key, value));
+    @Override public @NotNull CompletableFuture<Entry> getAndPut(@NotNull ByteArray key, @NotNull byte[] value) {
+        return metaStorageRaftGrpSvc.run(new GetAndPutCommand(key, value))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> putAll(@NotNull Map<Key, byte[]> vals) {
+    @Override public @NotNull CompletableFuture<Void> putAll(@NotNull Map<ByteArray, byte[]> vals) {
         return metaStorageRaftGrpSvc.run(new PutAllCommand(vals));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAndPutAll(@NotNull Map<Key, byte[]> vals) {
-        List<Key> keys = new ArrayList<>();
-        List<byte[]> values = new ArrayList<>();
-
-        vals.forEach((key, value) -> {
-            keys.add(key);
-            values.add(value);
-        });
-
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAndPutAllCommand(keys, values)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndPutAll(@NotNull Map<ByteArray, byte[]> vals) {
+        return metaStorageRaftGrpSvc.run(new GetAndPutAllCommand(vals)).
+            thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> remove(@NotNull Key key) {
+    @Override public @NotNull CompletableFuture<Void> remove(@NotNull ByteArray key) {
         return metaStorageRaftGrpSvc.run(new RemoveCommand(key));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull Key key) {
-        return metaStorageRaftGrpSvc.run(new GetAndRemoveCommand(key));
+    @Override public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull ByteArray key) {
+        return metaStorageRaftGrpSvc.run(new GetAndRemoveCommand(key))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<Key> keys) {
+    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Set<ByteArray> keys) {
         return metaStorageRaftGrpSvc.run(new RemoveAllCommand(keys));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAndRemoveAll(@NotNull Collection<Key> keys) {
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAndRemoveAllCommand(keys)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Set<ByteArray> keys) {
+        return metaStorageRaftGrpSvc.run(new GetAndRemoveAllCommand(keys)).
+            thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
-    // TODO: IGNITE-14389 Implement.
-    /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Boolean> invoke(@NotNull Condition condition,
-        @NotNull Collection<Operation> success, @NotNull Collection<Operation> failure) {
-        return null;
+    @Override public @NotNull CompletableFuture<Boolean> invoke(
+            @NotNull Condition condition,
+            @NotNull Operation success,
+            @NotNull Operation failure
+    ) {
+        return invoke(condition, List.of(success), List.of(failure));
     }
 
-    // TODO: IGNITE-14389 Either implement or remove this method.
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> getAndInvoke(@NotNull Key key, @NotNull Condition condition,
-        @NotNull Operation success, @NotNull Operation failure) {
-        return null;
+    @Override public @NotNull CompletableFuture<Boolean> invoke(
+            @NotNull Condition condition,
+            @NotNull Collection<Operation> success,
+            @NotNull Collection<Operation> failure
+    ) {
+        ConditionInfo cond = toConditionInfo(condition);
+
+        List<OperationInfo> successOps = toOperationInfos(success);
+
+        List<OperationInfo> failureOps = toOperationInfos(failure);
+
+        return metaStorageRaftGrpSvc.run(new InvokeCommand(cond, successOps, failureOps));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo, long revUpperBound) {
+    @Override public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound) {
         return new CursorImpl<>(
-            metaStorageRaftGrpSvc,
-            metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo, revUpperBound))
+                metaStorageRaftGrpSvc,
+                metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo, revUpperBound)),
+                MetaStorageServiceImpl::singleEntryResult
         );
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo) {
+    @Override public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo) {
         return new CursorImpl<>(
-            metaStorageRaftGrpSvc,
-            metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo))
+                metaStorageRaftGrpSvc,
+                metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo)),
+                MetaStorageServiceImpl::singleEntryResult
         );
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<IgniteUuid> watch(
-        @Nullable Key keyFrom,
-        @Nullable Key keyTo,
+        @Nullable ByteArray keyFrom,
+        @Nullable ByteArray keyTo,
         long revision,
         @NotNull WatchListener lsnr
     ) {
@@ -191,7 +204,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         watchRes.thenAccept(
             watchId -> watchProcessor.addWatch(
                 watchId,
-                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes),
+                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
                 lsnr
             )
         );
@@ -201,7 +214,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<IgniteUuid> watch(
-        @NotNull Key key,
+        @NotNull ByteArray key,
         long revision,
         @NotNull WatchListener lsnr
     ) {
@@ -210,7 +223,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<IgniteUuid> watch(
-        @NotNull Collection<Key> keys,
+        @NotNull Set<ByteArray> keys,
         long revision,
         @NotNull WatchListener lsnr
     ) {
@@ -220,7 +233,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         watchRes.thenAccept(
             watchId -> watchProcessor.addWatch(
                 watchId,
-                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes),
+                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
                 lsnr
             )
         );
@@ -233,12 +246,112 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         return CompletableFuture.runAsync(() -> watchProcessor.stopWatch(id));
     }
 
-    // TODO: IGNITE-14389 Implement.
+    // TODO: IGNITE-14734 Implement.
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Void> compact() {
         return null;
     }
 
+    /** */
+    private static List<OperationInfo> toOperationInfos(Collection<Operation> ops) {
+        List<OperationInfo> res = new ArrayList<>(ops.size());
+
+        for (Operation op : ops) {
+            OperationInfo info = null;
+
+            if (op.type() == OperationType.NO_OP)
+                info = new OperationInfo(null, null, OperationType.NO_OP);
+            else if (op.type() == OperationType.REMOVE)
+                info = new OperationInfo(((Operation.RemoveOp)op.inner()).key(), null, OperationType.REMOVE);
+            else if (op.type() == OperationType.PUT) {
+                Operation.PutOp inner = (Operation.PutOp)op.inner();
+
+                info = new OperationInfo(inner.key(), inner.value(), OperationType.PUT);
+            }
+            else
+                assert false : "Unknown operation type " + op.type();
+
+            res.add(info);
+        }
+
+        return res;
+    }
+
+    /** */
+    private static ConditionInfo toConditionInfo(@NotNull Condition condition) {
+        ConditionInfo cnd = null;
+
+        Object obj = condition.inner();
+
+        if (obj instanceof Condition.ExistenceCondition) {
+            Condition.ExistenceCondition inner = (Condition.ExistenceCondition)obj;
+
+            cnd = new ConditionInfo(inner.key(), inner.type(), null, 0);
+        }
+        else if (obj instanceof Condition.RevisionCondition) {
+            Condition.RevisionCondition inner = (Condition.RevisionCondition)obj;
+
+            cnd = new ConditionInfo(inner.key(), inner.type(), null, inner.revision());
+        }
+        else if (obj instanceof Condition.ValueCondition) {
+            Condition.ValueCondition inner = (Condition.ValueCondition)obj;
+
+            cnd = new ConditionInfo(inner.key(), inner.type(), inner.value(), 0);
+        }
+        else
+            assert false : "Unknown condition type: " + obj.getClass().getSimpleName();
+
+        return cnd;
+    }
+
+    /** */
+    private static Map<ByteArray, Entry> multipleEntryResult(Object obj) {
+        MultipleEntryResponse resp = (MultipleEntryResponse) obj;
+
+        Map<ByteArray, Entry> res = new HashMap<>();
+
+        for (SingleEntryResponse e : resp.entries()) {
+            ByteArray key = new ByteArray(e.key());
+
+            res.put(key, new EntryImpl(key, e.value(), e.revision(), e.updateCounter()));
+        }
+
+        return res;
+    }
+
+    /** */
+    private static Entry singleEntryResult(Object obj) {
+        SingleEntryResponse resp = (SingleEntryResponse) obj;
+
+        return new EntryImpl(new ByteArray(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(new ByteArray(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 {
@@ -247,7 +360,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
         /**
          * Starts exclusive thread per watch that implement watch pulling logic and
-         * calls {@link WatchListener#onUpdate(Iterable)}} or {@link WatchListener#onError(Throwable)}.
+         * calls {@link WatchListener#onUpdate(WatchEvent)}} or {@link WatchListener#onError(Throwable)}.
          *
          * @param watchId Watch id.
          * @param cursor Watch Cursor.
@@ -272,6 +385,10 @@ public class MetaStorageServiceImpl implements MetaStorageService {
                 (k, v) -> {
                     CompletableFuture.runAsync(v::interrupt).thenRun(() -> {
                         try {
+                            v.stop = true;
+
+                            Thread.sleep(100);
+
                             v.cursor.close();
                         }
                         catch (InterruptedException e) {
@@ -289,6 +406,9 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
         /** 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;
 
@@ -309,13 +429,9 @@ public class MetaStorageServiceImpl implements MetaStorageService {
              * in the while(true) loop. Collects watch events with same revision and fires either onUpdate or onError().
              */
             @Override public void run() {
-                long rev = -1;
-
-                List<WatchEvent> sameRevisionEvts = new ArrayList<>();
-
                 Iterator<WatchEvent> watchEvtsIter = cursor.iterator();
 
-                while (true) {
+                while (!stop) {
                     try {
                         if (watchEvtsIter.hasNext()) {
                             WatchEvent watchEvt = null;
@@ -329,19 +445,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
                             assert watchEvt != null;
 
-                            if (watchEvt.newEntry().revision() == rev)
-                                sameRevisionEvts.add(watchEvt);
-                            else {
-                                rev = watchEvt.newEntry().revision();
-
-                                if (!sameRevisionEvts.isEmpty()) {
-                                    lsnr.onUpdate(sameRevisionEvts);
-
-                                    sameRevisionEvts.clear();
-                                }
-
-                                sameRevisionEvts.add(watchEvt);
-                            }
+                            lsnr.onUpdate(watchEvt);
                         }
                         else
                             Thread.sleep(10);
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/CompactedException.java
similarity index 97%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java
copy to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/CompactedException.java
index 6ea9f43..c256442 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/CompactedException.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
 /**
  * Thrown when a requested operation on meta storage could not be performed because target revisions were removed
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java
new file mode 100644
index 0000000..f78b113
--- /dev/null
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java
@@ -0,0 +1,347 @@
+/*
+ * 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.metastorage.client;
+
+import java.util.Collection;
+import org.apache.ignite.internal.metastorage.common.ConditionType;
+
+/**
+ * Represents a condition for meta storage conditional update.
+ *
+ * @see MetaStorageService#invoke(Condition, Operation, Operation)
+ * @see MetaStorageService#invoke(Condition, Collection, Collection)
+ */
+public final class Condition {
+    /** Actual condition implementation. */
+    private final InnerCondition cond;
+
+    /**
+     * Constructs a condition which wraps the actual condition implementation.
+     *
+     * @param cond The actual condition implementation.
+     */
+    Condition(InnerCondition cond) {
+        this.cond = cond;
+    }
+
+    public InnerCondition inner() {
+        return cond;
+    }
+
+    public ConditionType type() {
+        return cond.type();
+    }
+
+    /**
+     * Represents condition on entry revision. Only one type of condition could be applied to
+     * the one instance of condition. Subsequent invocations of any method which produces condition will throw
+     * {@link IllegalStateException}.
+     */
+    public static final class RevisionCondition extends AbstractCondition {
+        /** The revision as the condition argument. */
+        private long rev;
+
+        /**
+         * Constructs a condition by a revision for an entry identified by the given key.
+         *
+         * @param key Identifies an entry which condition will be applied to.
+         */
+        RevisionCondition(byte[] key) {
+            super(key);
+        }
+
+        public long revision() {
+            return rev;
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_EQUAL}. This condition tests the given revision on equality with
+         * target entry revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition eq(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_EQUAL);
+
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_NOT_EQUAL}. This condition tests the given revision on inequality
+         * with target entry revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_NOT_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition ne(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_NOT_EQUAL);
+
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_GREATER}. This condition tests that the target entry revision
+         * is greater than given revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_GREATER}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition gt(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_GREATER);
+
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_GREATER_OR_EQUAL}. This condition tests that the target entry
+         * revision is greater than or equal to given revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_GREATER_OR_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition ge(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_GREATER_OR_EQUAL);
+
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_LESS}. This condition tests that target entry revision
+         * is less than the given revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_LESS}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition lt(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_LESS);
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#REV_LESS_OR_EQUAL}. This condition tests that target entry revision
+         * is less than or equal to the given revision.
+         *
+         * @param rev The revision.
+         * @return The condition of type {@link ConditionType#REV_LESS_OR_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition le(long rev) {
+            validate(type());
+
+            type(ConditionType.REV_LESS_OR_EQUAL);
+
+            this.rev = rev;
+
+            return new Condition(this);
+        }
+    }
+
+    /**
+     * Represents condition on entry value. Only one type of condition could be applied to
+     * the one instance of condition. Subsequent invocations of any method which produces condition will throw
+     * {@link IllegalStateException}.
+     */
+    public static final class ValueCondition extends AbstractCondition {
+        /** The value as the condition argument. */
+        private byte[] val;
+
+        /**
+         * Constructs a condition by a value for an entry identified by the given key.
+         *
+         * @param key Identifies an entry which condition will be applied to.
+         */
+        ValueCondition(byte[] key) {
+            super(key);
+        }
+
+        public byte[] value() {
+            return val;
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#VAL_EQUAL}. This condition tests the given value on equality with
+         * target entry value.
+         *
+         * @param val The value.
+         * @return The condition of type {@link ConditionType#VAL_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition eq(byte[] val) {
+            validate(type());
+
+            type(ConditionType.VAL_EQUAL);
+
+            this.val = val;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#VAL_NOT_EQUAL}. This condition tests the given value on inequality
+         * with target entry value.
+         *
+         * @param val The value.
+         * @return The condition of type {@link ConditionType#VAL_NOT_EQUAL}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition ne(byte[] val) {
+            validate(type());
+
+            type(ConditionType.VAL_NOT_EQUAL);
+
+            this.val = val;
+
+            return new Condition(this);
+        }
+    }
+
+    /**
+     * Represents condition on an entry existence. Only one type of condition could be applied to
+     * the one instance of condition. Subsequent invocations of any method which produces condition will throw
+     * {@link IllegalStateException}.
+     */
+    public static final class ExistenceCondition extends AbstractCondition {
+        /**
+         * Constructs a condition on existence an entry identified by the given key.
+         *
+         * @param key Identifies an entry which condition will be applied to.
+         */
+        ExistenceCondition(byte[] key) {
+            super(key);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#KEY_EXISTS}. This condition tests the existence of an entry
+         * identified by the given key.
+         *
+         * @return The condition of type {@link ConditionType#KEY_EXISTS}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition exists() {
+            validate(type());
+
+            type(ConditionType.KEY_EXISTS);
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link ConditionType#KEY_NOT_EXISTS}. This condition tests the non-existence of an entry
+         * identified by the given key.
+         *
+         * @return The condition of type {@link ConditionType#KEY_NOT_EXISTS}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition notExists() {
+            validate(type());
+
+            type(ConditionType.KEY_NOT_EXISTS);
+
+            return new Condition(this);
+        }
+    }
+
+    /**
+     * Checks that condition is not defined yet. If the condition is already defined then exception will be thrown.
+     *
+     * @throws IllegalStateException In case when the condition is already defined.
+     */
+    private static void validate(Enum<?> type) {
+        if (type != null)
+            throw new IllegalStateException("Condition type " + type.name() + " is already defined.");
+    }
+
+    /**
+     * Defines condition interface.
+     */
+    private interface InnerCondition {
+        /**
+         * Returns key which identifies an entry which condition will be applied to.
+         *
+         * @return Key which identifies an entry which condition will be applied to.
+         */
+        byte[] key();
+
+        ConditionType type();
+    }
+
+    /**
+     * Defines an abstract condition with the key which identifies an entry which condition will be applied to.
+     */
+    private abstract static class AbstractCondition implements InnerCondition {
+        /** Entry key. */
+        private final byte[] key;
+
+        /**
+         * Condition type.
+         */
+        private ConditionType type;
+
+        /**
+         * Constructs a condition with the given entry key.
+         *
+         * @param key Key which identifies an entry which condition will be applied to.
+         */
+        private AbstractCondition(byte[] key) {
+            this.key = key;
+        }
+
+        /**
+         * Returns the key which identifies an entry which condition will be applied to.
+         *
+         * @return Key which identifies an entry which condition will be applied to.
+         */
+        @Override public byte[] key() {
+            return key;
+        }
+
+        @Override public ConditionType type() {
+            return type;
+        }
+
+        protected void type(ConditionType type) {
+            this.type = type;
+        }
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Conditions.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Conditions.java
similarity index 54%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Conditions.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Conditions.java
index 87fa238..3d4f370 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Conditions.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Conditions.java
@@ -15,7 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
+
+import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * This class contains fabric methods which produce conditions needed for conditional multi update functionality
@@ -24,47 +27,46 @@ package org.apache.ignite.metastorage.common;
  * @see Condition
  */
 public final class Conditions {
-
-    /** Key. */
-    private Key key;
-
-    /**
-     * Creates new condition for entry with concrete key.
-     *
-     * @param key Key
-     */
-    private Conditions(Key key) {
-        this.key = key;
-    }
-
     /**
      * Creates condition on entry revision.
      *
+     * @param key Identifies an entry which condition will be applied to. Can't be {@code null}.
      * @return Condition on entry revision.
      * @see Condition.RevisionCondition
      */
-    public Condition.RevisionCondition revision() {
-        return new Condition.RevisionCondition(key);
+    public static Condition.RevisionCondition revision(@NotNull ByteArray key) {
+        return new Condition.RevisionCondition(key.bytes());
     }
 
     /**
      * Creates condition on entry value.
      *
+     * @param key Identifies an entry which condition will be applied to. Can't be {@code null}.
      * @return Condition on entry value.
      * @see Condition.ValueCondition
      */
-    public Condition.ValueCondition value() {
-        return new Condition.ValueCondition(key);
+    public static Condition.ValueCondition value(@NotNull ByteArray key) {
+        return new Condition.ValueCondition(key.bytes());
+    }
+
+    /**
+     * Creates condition on entry existence.
+     *
+     * @param key Identifies an entry which condition will be applied to. Can't be {@code null}.
+     * @return Condition on entry existence.
+     */
+    public static Condition exists(@NotNull ByteArray key) {
+        return new Condition.ExistenceCondition(key.bytes()).exists();
     }
 
     /**
-     * Creates key-based condition.
+     * Creates condition on entry not existence.
      *
-     * @param key Key of condition.
-     * @return Key-based condition instance.
+     * @param key Identifies an entry which condition will be applied to. Can't be {@code null}.
+     * @return Condition on entry not existence.
      */
-    public static Conditions key(Key key) {
-        return new Conditions(key);
+    public static Condition notExists(@NotNull ByteArray key) {
+        return new Condition.ExistenceCondition(key.bytes()).notExists();
     }
 
     /**
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Entry.java
similarity index 77%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java
copy to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Entry.java
index 7f653cc..8895aac 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Entry.java
@@ -15,8 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
+import org.apache.ignite.lang.ByteArray;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -34,7 +35,7 @@ public interface Entry {
      *
      * @return The key.
      */
-    @NotNull Key key();
+    @NotNull ByteArray key();
 
     /**
      * Returns a value. Could be {@code null} for empty entry.
@@ -56,4 +57,18 @@ public interface Entry {
      * @return Update counter.
      */
     long updateCounter();
+
+    /**
+     * Returns value which denotes whether entry is empty or not.
+     *
+     * @return {@code True} if entry is empty, otherwise - {@code false}.
+     */
+    boolean empty();
+
+    /**
+     * Returns value which denotes whether entry is tombstone or not.
+     *
+     * @return {@code True} if entry is tombstone, otherwise - {@code false}.
+     */
+    boolean tombstone();
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchEvent.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/EntryEvent.java
similarity index 81%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchEvent.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/EntryEvent.java
index 119167a..9e9a70d 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchEvent.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/EntryEvent.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
 import java.io.Serializable;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Watch event which can be processed by {@link WatchListener}.
+ * Represent an update event for particular key and entry.
  */
-public final class WatchEvent implements Serializable {
-    /** Old (previous) entry */
+public final class EntryEvent implements Serializable {
+    /** Old (previous) entry. */
     @NotNull private final Entry oldEntry;
 
     /** New (updated) entry. */
@@ -34,9 +34,9 @@ public final class WatchEvent implements Serializable {
      * Constructs an event with given old and new entries.
      *
      * @param oldEntry Old entry.
-     * @param newEntry New entry/
+     * @param newEntry New entry.
      */
-    public WatchEvent(@NotNull Entry oldEntry, @NotNull Entry newEntry) {
+    public EntryEvent(Entry oldEntry, Entry newEntry) {
         this.oldEntry = oldEntry;
         this.newEntry = newEntry;
     }
@@ -46,7 +46,8 @@ public final class WatchEvent implements Serializable {
      *
      * @return Old entry.
      */
-    public @NotNull Entry oldEntry() {
+    @NotNull
+    public Entry oldEntry() {
         return oldEntry;
     }
 
@@ -55,7 +56,8 @@ public final class WatchEvent implements Serializable {
      *
      * @return New entry.
      */
-    public @NotNull Entry newEntry() {
+    @NotNull
+    public Entry newEntry() {
         return newEntry;
     }
 
@@ -63,20 +65,24 @@ public final class WatchEvent implements Serializable {
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;
+
         if (o == null || getClass() != o.getClass())
             return false;
 
-        WatchEvent that = (WatchEvent)o;
+        EntryEvent that = (EntryEvent)o;
 
         if (!oldEntry.equals(that.oldEntry))
             return false;
+
         return newEntry.equals(that.newEntry);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
         int res = oldEntry.hashCode();
+
         res = 31 * res + newEntry.hashCode();
+
         return res;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Key.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Key.java
similarity index 98%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Key.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Key.java
index 7cba39f..58d416e 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Key.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Key.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
 import java.io.Serializable;
 import java.nio.charset.StandardCharsets;
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/MetaStorageService.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/MetaStorageService.java
index 6a13637..787a463 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/MetaStorageService.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/MetaStorageService.java
@@ -19,16 +19,11 @@ package org.apache.ignite.metastorage.client;
 
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.CompactedException;
-import org.apache.ignite.metastorage.common.Condition;
-import org.apache.ignite.metastorage.common.Cursor;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.Operation;
-import org.apache.ignite.metastorage.common.OperationTimeoutException;
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -42,57 +37,57 @@ public interface MetaStorageService {
      * @param key Key. Couldn't be {@code null}.
      * @return An 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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Entry> get(@NotNull Key key);
+    CompletableFuture<Entry> get(@NotNull ByteArray key);
 
     /**
      * Retrieves an entry for the given key and the revision upper bound.
      *
      * @param key The key. Couldn't be {@code null}.
-     * @param revUpperBound  The upper bound for entry revisions. Must be positive.
+     * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Entry> get(@NotNull Key key, long revUpperBound);
+    CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound);
 
     /**
      * Retrieves entries for given keys.
      *
-     * @param keys The collection of keys. Couldn't be {@code null} or empty.
-     *             Collection 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}.
      * @return A map of entries for given keys. Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys);
+    CompletableFuture<Map<ByteArray, Entry>> getAll(Set<ByteArray> keys);
 
     /**
      * Retrieves entries for given keys and the revision upper bound.
      *
-     * @param keys The collection of keys. Couldn't be {@code null} or empty.
-     *             Collection elements couldn't be {@code null}.
-     * @param revUpperBound  The upper bound for entry revisions. Must be positive.
+     * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys, long revUpperBound);
+    CompletableFuture<Map<ByteArray, Entry>> getAll(Set<ByteArray> keys, long revUpperBound);
 
     /**
      * Inserts or updates an entry with the given key and the given value.
@@ -101,11 +96,11 @@ public interface MetaStorageService {
      * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Void> put(@NotNull Key key, @NotNull byte[] value);
+    CompletableFuture<Void> put(@NotNull ByteArray key, @NotNull byte[] value);
 
     /**
      * Inserts or updates an entry with the given key and the given value and
@@ -115,11 +110,11 @@ public interface MetaStorageService {
      * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Entry> getAndPut(@NotNull Key key, @NotNull byte[] value);
+    CompletableFuture<Entry> getAndPut(@NotNull ByteArray key, @NotNull byte[] value);
 
     /**
      * Inserts or updates entries with given keys and given values.
@@ -127,11 +122,11 @@ public interface MetaStorageService {
      * @param vals The map of keys and corresponding values. Couldn't be {@code null} or empty.
      * @return Completed future.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Void> putAll(@NotNull Map<Key, byte[]> vals);
+    CompletableFuture<Void> putAll(@NotNull Map<ByteArray, byte[]> vals);
 
     /**
      * Inserts or updates entries with given keys and given values and
@@ -140,11 +135,11 @@ public interface MetaStorageService {
      * @param vals The map of keys and corresponding values. Couldn't be {@code null} or empty.
      * @return A map of entries for given keys. Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Map<Key, Entry>> getAndPutAll(@NotNull Map<Key, byte[]> vals);
+    CompletableFuture<Map<ByteArray, Entry>> getAndPutAll(@NotNull Map<ByteArray, byte[]> vals);
 
     /**
      * Removes an entry for the given key.
@@ -152,11 +147,11 @@ public interface MetaStorageService {
      * @param key The key. Couldn't be {@code null}.
      * @return Completed future.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Void> remove(@NotNull Key key);
+    CompletableFuture<Void> remove(@NotNull ByteArray key);
 
     /**
      * Removes an entry for the given key.
@@ -164,23 +159,23 @@ public interface MetaStorageService {
      * @param key The key. 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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Entry> getAndRemove(@NotNull Key key);
+    CompletableFuture<Entry> getAndRemove(@NotNull ByteArray key);
 
     /**
      * Removes entries for given keys.
      *
-     * @param keys The keys collection. Couldn't be {@code null}.
+     * @param keys The keys set. Couldn't be {@code null} or empty.
      * @return Completed future.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Void> removeAll(@NotNull Collection<Key> keys);
+    CompletableFuture<Void> removeAll(@NotNull Set<ByteArray> keys);
 
     /**
      * Removes entries for given keys and retrieves previous entries.
@@ -190,12 +185,11 @@ public interface MetaStorageService {
      * The order of entries in the result list corresponds to the traversal order of {@code keys} collection.
      * Couldn't be {@code null}.
      * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Map<Key, Entry>> getAndRemoveAll(@NotNull Collection<Key> keys);
-
+    CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Set<ByteArray> keys);
 
     /**
      * Updates an entry for the given key conditionally.
@@ -203,11 +197,11 @@ 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 Batch of updates which will be atomically applied in case of condition evaluation yields {@code true}.
-     * @param failure Batch of updates which will be atomically applied in case of condition evaluation yields {@code false}.
-     * @return Future result {@code true} if {@code success} updates were applied, otherwise {@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 Key
+     * @see ByteArray
      * @see Entry
      * @see Condition
      * @see Operation
@@ -215,28 +209,27 @@ public interface MetaStorageService {
     // TODO: https://issues.apache.org/jira/browse/IGNITE-14269: will be replaced by conditional multi update.
     @NotNull
     CompletableFuture<Boolean> invoke(@NotNull Condition condition,
-        @NotNull Collection<Operation> success, @NotNull Collection<Operation> failure);
+                                      @NotNull Operation success, @NotNull Operation failure);
 
     /**
      * Updates an entry for the given key conditionally.
      *
      * <p>Conditional update could be treated as <i>if(condition)-then(success)-else(failure)</i> expression.</p>
      *
-     * @param key The key. Couldn't be {@code null}.
      * @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}.
-     * @return A previous entry for the given key.
+     * @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 Key
+     * @see ByteArray
      * @see Entry
      * @see Condition
      * @see Operation
      */
-    //TODO: https://issues.apache.org/jira/browse/IGNITE-14269: will be replaced by conditional multi update.
+    // TODO: https://issues.apache.org/jira/browse/IGNITE-14269: will be replaced by conditional multi update.
     @NotNull
-    CompletableFuture<Entry> getAndInvoke(@NotNull Key key, @NotNull Condition condition,
-                                          @NotNull Operation success, @NotNull Operation failure);
+    CompletableFuture<Boolean> invoke(@NotNull Condition condition,
+                                      @NotNull Collection<Operation> success, @NotNull Collection<Operation> failure);
 
     /**
      * Retrieves entries for the given key range in lexicographic order. Entries will be filtered out by upper bound
@@ -248,26 +241,26 @@ public interface MetaStorageService {
      * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo, long revUpperBound);
+    Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound);
 
     /**
      * Retrieves entries for the given key range in lexicographic order. Short cut for
-     * {@link #range(Key, Key, long)} where {@code revUpperBound == -1}.
+     * {@link #range(ByteArray, ByteArray, long)} 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}.
      * @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.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo);
+    Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo);
 
     /**
      * Subscribes on meta storage updates matching the parameters.
@@ -281,11 +274,11 @@ public interface MetaStorageService {
      * @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 Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<IgniteUuid> watch(@Nullable Key keyFrom, @Nullable Key keyTo,
+    CompletableFuture<IgniteUuid> watch(@Nullable ByteArray keyFrom, @Nullable ByteArray keyTo,
                                   long revision, @NotNull WatchListener lsnr);
 
     /**
@@ -299,16 +292,16 @@ public interface MetaStorageService {
      * @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 Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<IgniteUuid> watch(@NotNull Key key, long revision, @NotNull WatchListener lsnr);
+    CompletableFuture<IgniteUuid> watch(@NotNull ByteArray key, long revision, @NotNull WatchListener lsnr);
 
     /**
      * Subscribes on meta storage updates for given keys.
      *
-     * @param keys Collection of target keys. Could be {@code null}.
+     * @param keys Set of target keys. Could be {@code null}.
      * @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.
@@ -316,11 +309,11 @@ public interface MetaStorageService {
      * @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 Key
+     * @see ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<IgniteUuid> watch(@NotNull Collection<Key> keys, long revision, @NotNull WatchListener lsnr);
+    CompletableFuture<IgniteUuid> watch(@NotNull Set<ByteArray> keys, long revision, @NotNull WatchListener lsnr);
 
     /**
      * Cancels subscription for the given identifier.
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java
new file mode 100644
index 0000000..cf7a43e
--- /dev/null
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java
@@ -0,0 +1,166 @@
+/*
+ * 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.metastorage.client;
+
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines operation for meta storage conditional update (invoke).
+ */
+public final class Operation {
+    /** Actual operation implementation. */
+    private final InnerOp upd;
+
+    /**
+     * Constructs an operation which wraps the actual operation implementation.
+     *
+     * @param upd The actual operation implementation.
+     */
+    Operation(InnerOp upd) {
+        this.upd = upd;
+    }
+
+    /** */
+    public InnerOp inner() {
+        return upd;
+    }
+
+    /**
+     * Returns operation type.
+     *
+     * @return Operation type.
+     */
+    public OperationType type() {
+        return upd.type();
+    }
+
+    /**
+     * Represents operation of type <i>remove</i>.
+     */
+    public static final class RemoveOp extends AbstractOp {
+        /**
+         * Default no-op constructor.
+         *
+         * @param key Identifies an entry which operation will be applied to.
+         */
+        RemoveOp(byte[] key) {
+            super(key, OperationType.REMOVE);
+        }
+    }
+
+    /**
+     * Represents operation of type <i>put</i>.
+     */
+    public static final class PutOp extends AbstractOp {
+        /** Value. */
+        private final byte[] val;
+
+        /**
+         * Constructs operation of type <i>put</i>.
+         *
+         * @param key Identifies an entry which operation will be applied to.
+         * @param val The value to which the entry should be updated.
+         */
+        PutOp(byte[] key, byte[] val) {
+            super(key, OperationType.PUT);
+
+            this.val = val;
+        }
+
+        /**
+         * Returns value.
+         *
+         * @return Value.
+         */
+        public byte[] value() {
+            return val;
+        }
+    }
+
+    /**
+     * Represents operation of type <i>no-op</i>.
+     */
+    public static final class NoOp extends AbstractOp {
+        /**
+         * Default no-op constructor.
+         */
+        NoOp() {
+            super(null, OperationType.NO_OP);
+        }
+    }
+
+    /**
+     * Defines operation interface.
+     */
+    private interface InnerOp {
+        /**
+         * Returns key.
+         *
+         * @return Key.
+         */
+        @Nullable byte[] key();
+
+        /**
+         * Returns operation type.
+         *
+         * @return Operation type.
+         */
+        @NotNull OperationType type();
+    }
+
+    /** */
+    private static class AbstractOp implements InnerOp {
+        /** Key. */
+        @Nullable private final byte[] key;
+
+        /** Operation type. */
+        @NotNull private final OperationType type;
+
+        /**
+         * Ctor.
+         * @param key Key.
+         * @param type Operation type.
+         */
+        private AbstractOp(@Nullable byte[] key, OperationType type) {
+            this.key = key;
+            this.type = type;
+        }
+
+        /**
+         * Returns key.
+         *
+         * @return Key.
+         */
+        @Nullable
+        @Override public byte[] key() {
+            return key;
+        }
+
+        /**
+         * Returns operation type.
+         *
+         * @return Operation type.
+         */
+        @NotNull
+        @Override public OperationType type() {
+            return type;
+        }
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/OperationTimeoutException.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/OperationTimeoutException.java
similarity index 97%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/OperationTimeoutException.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/OperationTimeoutException.java
index 4edcf02..3428f39 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/OperationTimeoutException.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/OperationTimeoutException.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
 /**
  * Thrown when an operation is not executed within a specified time period. Usually in such cases the operation
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operations.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operations.java
similarity index 79%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operations.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operations.java
index 31c7449..87f806f 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operations.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operations.java
@@ -15,7 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
+
+import org.apache.ignite.lang.ByteArray;
 
 /**
  * This class contains fabric methods which produce operations needed for conditional multi update functionality
@@ -30,20 +32,22 @@ public final class Operations {
     /**
      * Creates operation of type <i>remove</i>. This type of operation removes entry.
      *
+     * @param key Identifies an entry which operation will be applied to.
      * @return Operation of type <i>remove</i>.
      */
-    public static Operation remove(Key key) {
-        return new Operation(new Operation.RemoveOp(key));
+    public static Operation remove(ByteArray key) {
+        return new Operation(new Operation.RemoveOp(key.bytes()));
     }
 
     /**
      * Creates operation of type <i>put</i>. This type of operation inserts or updates value of entry.
      *
+     * @param key Identifies an entry which operation will be applied to.
      * @param value Value.
      * @return Operation of type <i>put</i>.
      */
-    public static Operation put(Key key, byte[] value) {
-        return new Operation(new Operation.PutOp(key, value));
+    public static Operation put(ByteArray key, byte[] value) {
+        return new Operation(new Operation.PutOp(key.bytes(), value));
     }
 
     /**
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java
new file mode 100644
index 0000000..147ff63
--- /dev/null
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java
@@ -0,0 +1,82 @@
+/*
+ * 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.metastorage.client;
+
+import java.util.Collection;
+import java.util.List;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Watch event contains all entry updates done under one revision. Each particular entry update in this revision
+ * is represented by {@link EntryEvent} entity.
+ */
+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;
+
+    /**
+     * Constructs an watch event with given entry events collection.
+     *
+     * @param entryEvts Events for entries corresponding to an update under one revision.
+     */
+    public WatchEvent(List<EntryEvent> entryEvts) {
+        assert entryEvts != null && !entryEvts.isEmpty();
+
+        this.single = entryEvts.size() == 1;
+        this.entryEvts = entryEvts;
+    }
+
+    /**
+     * Constructs watch event with single entry update.
+     *
+     * @param entryEvt Entry event.
+     */
+    public WatchEvent(@NotNull EntryEvent entryEvt) {
+        this(List.of(entryEvt));
+    }
+
+    /**
+     * Returns {@code true} if watch event contains only one entry event.
+     *
+     * @return {@code True} if watch event contains only one entry event.
+     */
+    public boolean single() {
+        return single;
+    }
+
+    /**
+     * Returns collection of entry entry event done under one revision.
+     *
+     * @return Collection of entry entry event done under one revision.
+     */
+    public Collection<EntryEvent> entryEvents() {
+        return entryEvts;
+    }
+
+    /**
+     * Returns entry event. It is useful method in case when we know that only one event was modified.
+     *
+     * @return Entry event.
+     */
+    public EntryEvent entryEvent() {
+        return entryEvts.get(0);
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchListener.java
similarity index 87%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
copy to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchListener.java
index 1ed196e..37c6a6f 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchListener.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
 import org.jetbrains.annotations.NotNull;
 
@@ -26,11 +26,11 @@ public interface WatchListener {
     /**
      * The method will be called on each meta storage update.
      *
-     * @param events A single event or a batch. The batch always contains updates for specific revision.
+     * @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.
      */
-    boolean onUpdate(@NotNull Iterable<WatchEvent> events);
+    boolean onUpdate(@NotNull WatchEvent evt);
 
     /**
      * The method will be called in case of an error occurred. The listener and corresponding watch will be
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/ConditionType.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/ConditionType.java
new file mode 100644
index 0000000..522bd8f
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/ConditionType.java
@@ -0,0 +1,53 @@
+/*
+ * 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.common;
+
+/**
+ * Defines possible condition types which can be applied to the revision.
+ */
+public enum ConditionType {
+    /** Equality condition type for revision. */
+    REV_EQUAL,
+
+    /** Inequality condition type for revision. */
+    REV_NOT_EQUAL,
+
+    /** Greater than condition type for revision. */
+    REV_GREATER,
+
+    /** Less than condition type for revision. */
+    REV_LESS,
+
+    /** Less than or equal to condition type for revision. */
+    REV_LESS_OR_EQUAL,
+
+    /** Greater than or equal to condition type for revision. */
+    REV_GREATER_OR_EQUAL,
+
+    /** Equality condition type for value. */
+    VAL_EQUAL,
+
+    /** Inequality condition type for value. */
+    VAL_NOT_EQUAL,
+
+    /** Existence condition type for key. */
+    KEY_EXISTS,
+
+    /** Non-existence condition type for key. */
+    KEY_NOT_EXISTS
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/OperationType.java
similarity index 76%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
copy to modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/OperationType.java
index dcc1c94..7c97859 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/OperationType.java
@@ -15,13 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.metastorage.common;
 
 /**
- * Closeable cursor.
- *
- * @param <T>
+ * Defines possible operation types.
  */
-//TODO: Should be replaced by common entity from org.ignite.lang or org.ignite.core package.
-public interface Cursor<T> extends Iterable<T>, AutoCloseable {
+public enum OperationType {
+    /** No-op operation. */
+    NO_OP,
+
+    /** Put (insert/replace) operation. */
+    PUT,
+
+    /** Remove operation. */
+    REMOVE
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ConditionInfo.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ConditionInfo.java
new file mode 100644
index 0000000..998be42
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ConditionInfo.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.common.command;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.metastorage.common.ConditionType;
+
+/**
+ * Defines condition for InvokeCommand.
+ */
+public class ConditionInfo implements Serializable {
+    /** Key. */
+    private final byte[] key;
+
+    /** Type. */
+    private final ConditionType type;
+
+    /** Value. */
+    private final byte[] val;
+
+    /** Revision. */
+    private final long rev;
+
+    /**
+     * Construct condition with given parameters.
+     *
+     * @param key Key.
+     * @param type Condition type.
+     * @param val Value.
+     * @param rev Revision.
+     */
+    public ConditionInfo(byte[] key, ConditionType type, byte[] val, long rev) {
+        this.key = key;
+        this.type = type;
+        this.val = val;
+        this.rev = rev;
+    }
+
+    /**
+     * Returns key.
+     *
+     * @return Key.
+     */
+    public byte[] key() {
+        return key;
+    }
+
+    /**
+     * Returns condition type.
+     *
+     * @return Condition type.
+     */
+    public ConditionType type() {
+        return type;
+    }
+
+    /**
+     * Returns value.
+     *
+     * @return Value.
+     */
+    public byte[] value() {
+        return val;
+    }
+
+    /**
+     * Returns revision.
+     *
+     * @return Revision.
+     */
+    public long revision() {
+        return rev;
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ErrorResponse.java
similarity index 60%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
rename to modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ErrorResponse.java
index dcc1c94..8ef1744 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ErrorResponse.java
@@ -15,13 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.metastorage.common.command;
 
-/**
- * Closeable cursor.
- *
- * @param <T>
- */
-//TODO: Should be replaced by common entity from org.ignite.lang or org.ignite.core package.
-public interface Cursor<T> extends Iterable<T>, AutoCloseable {
+import java.io.Serializable;
+
+/** Defines error response. */
+public class ErrorResponse implements Serializable {
+    /** Error code. */
+    private final int errCode;
+
+    /**
+     * Constructs error response.
+     *
+     * @param errCode Error code
+     */
+    public ErrorResponse(int errCode) {
+        this.errCode = errCode;
+    }
+
+    /**
+     * Returns error code.
+     *
+     * @return Error code.
+     */
+    public int errorCode() {
+        return errCode;
+    }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAllCommand.java
index 227ba38..c129931 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAllCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAllCommand.java
@@ -17,37 +17,35 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Get all command for {@link MetaStorageCommandListener} that retrieves entries
+ * Get all command for MetaStorageCommandListener that retrieves entries
  * for given keys and the revision upper bound, if latter is present.
  */
 public final class GetAllCommand implements ReadCommand {
-    /** The collection of keys. */
-    @NotNull private final Collection<Key> keys;
+    /** The list of keys. */
+    @NotNull private final List<byte[]> keys;
 
     /** The upper bound for entry revisions. Must be positive. */
-    @Nullable private Long revUpperBound;
+    private long revUpperBound;
 
     /**
      * @param keys The collection of keys. Couldn't be {@code null} or empty. Collection elements couldn't be {@code
      * null}.
      */
-    public GetAllCommand(@NotNull Collection<Key> keys) {
+    public GetAllCommand(@NotNull Set<ByteArray> keys) {
         assert !keys.isEmpty();
 
-        if (keys instanceof Serializable)
-            this.keys = keys;
-        else
-            this.keys = new ArrayList<>(keys);
+        this.keys = new ArrayList<>(keys.size());
+
+        for (ByteArray key : keys)
+            this.keys.add(key.bytes());
     }
 
     /**
@@ -55,7 +53,7 @@ public final class GetAllCommand implements ReadCommand {
      * null}.
      * @param revUpperBound The upper bound for entry revisions. Must be positive.
      */
-    public GetAllCommand(@NotNull Collection<Key> keys, @NotNull Long revUpperBound) {
+    public GetAllCommand(@NotNull Set<ByteArray> keys, long revUpperBound) {
         this(keys);
 
         assert revUpperBound > 0;
@@ -64,16 +62,16 @@ public final class GetAllCommand implements ReadCommand {
     }
 
     /**
-     * @return The collection of keys.
+     * @return The list of keys.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull List<byte[]> keys() {
         return keys;
     }
 
     /**
      * @return The upper bound for entry revisions. Must be positive.
      */
-    public @Nullable Long revision() {
+    public long revision() {
         return revUpperBound;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutAllCommand.java
index d5bc3e5..cf11013 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutAllCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutAllCommand.java
@@ -17,40 +17,44 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import java.util.Map;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Get and put all command for {@link MetaStorageCommandListener} that inserts or updates entries
+ * Get and put all command for MetaStorageCommandListener that inserts or updates entries
  * with given keys and given values and retrieves a previous entries for given keys.
  */
 public final class GetAndPutAllCommand implements WriteCommand {
     /** Keys. */
-    @NotNull private final List<Key> keys;
+    @NotNull private final List<byte[]> keys;
 
     /** Values. */
     @NotNull private final List<byte[]> vals;
 
     /**
-     * @param keys Keys.
      * @param vals Values.
      */
-    public GetAndPutAllCommand(@NotNull List<Key> keys, @NotNull List<byte[]> vals) {
-        assert keys instanceof Serializable;
-        assert vals instanceof Serializable;
+    public GetAndPutAllCommand(@NotNull Map<ByteArray, byte[]> vals) {
+        int size = vals.size();
 
-        this.keys = keys;
-        this.vals = vals;
+        this.keys = new ArrayList<>(size);
+        this.vals = new ArrayList<>(size);
+
+        for (Map.Entry<ByteArray, byte[]> e : vals.entrySet()) {
+            this.keys.add(e.getKey().bytes());
+
+            this.vals.add(e.getValue());
+        }
     }
 
     /**
      * @return Keys.
      */
-    public @NotNull List<Key> keys() {
+    public @NotNull List<byte[]> keys() {
         return keys;
     }
 
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutCommand.java
index 95581ad..26bfbe5 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndPutCommand.java
@@ -17,18 +17,17 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Get and put command for {@link MetaStorageCommandListener} that inserts or updates an entry
+ * Get and put command for MetaStorageCommandListener that inserts or updates an entry
  * with the given key and the given value and retrieves a previous entry for the given key.
  */
 public final class GetAndPutCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+    @NotNull private final byte[] key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,15 +36,15 @@ public final class GetAndPutCommand implements WriteCommand {
      * @param key The key. Couldn't be {@code null}.
      * @param val The value. Couldn't be {@code null}.
      */
-    public GetAndPutCommand(@NotNull Key key, @NotNull byte[] val) {
-        this.key = key;
+    public GetAndPutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
+        this.key = key.bytes();
         this.val = val;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull byte[] key() {
         return key;
     }
 
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveAllCommand.java
index bea5d1a..0ff8738 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveAllCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveAllCommand.java
@@ -17,38 +17,35 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Get and remove all command for {@link MetaStorageCommandListener} that removes entries
+ * Get and remove all command for MetaStorageCommandListener that removes entries
  * for given keys and retrieves previous entries.
  */
 public final class GetAndRemoveAllCommand implements WriteCommand {
     /** The keys collection. Couldn't be {@code null}. */
-    @NotNull private final Collection<Key> keys;
+    @NotNull private final List<byte[]> keys;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      */
-    public GetAndRemoveAllCommand(@NotNull Collection<Key> keys) {
-        assert !keys.isEmpty();
+    public GetAndRemoveAllCommand(@NotNull Set<ByteArray> keys) {
+        this.keys = new ArrayList<>(keys.size());
 
-        if (keys instanceof Serializable)
-            this.keys = keys;
-        else
-            this.keys = new ArrayList<>(keys);
+        for (ByteArray key : keys)
+            this.keys.add(key.bytes());
     }
 
     /**
      * @return The keys collection. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull List<byte[]> keys() {
         return keys;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveCommand.java
index 066fc48..57288ad 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetAndRemoveCommand.java
@@ -17,30 +17,29 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Get and remove command for {@link MetaStorageCommandListener} that removes an entry
+ * Get and remove command for MetaStorageCommandListener that removes an entry
  * for the given key and retrieves a previous entry for the given key.
  */
 public final class GetAndRemoveCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+    @NotNull private final byte[] key;
 
     /**
      * @param key The key. Couldn't be {@code null}.
      */
-    public GetAndRemoveCommand(@NotNull Key key) {
-        this.key = key;
+    public GetAndRemoveCommand(@NotNull ByteArray key) {
+        this.key = key.bytes();
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull byte[] key() {
         return key;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetCommand.java
index e3468ec..381a8aa 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetCommand.java
@@ -17,36 +17,34 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Get command for {@link MetaStorageCommandListener} that retrieves an entry
+ * Get command for MetaStorageCommandListener that retrieves an entry
  * for the given key and the revision upper bound, if latter is present.
  */
 public final class GetCommand implements ReadCommand {
     /** Key. */
-    @NotNull private final Key key;
+    @NotNull private final byte[] key;
 
     /** The upper bound for entry revisions. Must be positive. */
-    @Nullable private Long revUpperBound;
+    private long revUpperBound;
 
     /**
      * @param key Key. Couldn't be {@code null}.
      */
-    public GetCommand(@NotNull Key key) {
-        this.key = key;
+    public GetCommand(@NotNull ByteArray key) {
+        this.key = key.bytes();
     }
 
     /**
      * @param key Key. Couldn't be {@code null}.
      * @param revUpperBound The upper bound for entry revisions. Must be positive.
      */
-    public GetCommand(@NotNull Key key, @NotNull Long revUpperBound) {
-        this.key = key;
+    public GetCommand(@NotNull ByteArray key, long revUpperBound) {
+        this.key = key.bytes();
 
         assert revUpperBound > 0;
 
@@ -56,14 +54,14 @@ public final class GetCommand implements ReadCommand {
     /**
      * @return Key.
      */
-    public @NotNull Key key() {
+    public @NotNull byte[] key() {
         return key;
     }
 
     /**
      * @return The upper bound for entry revisions, or {@code null} if wasn't specified.
      */
-    public @Nullable Long revision() {
+    public long revision() {
         return revUpperBound;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/InvokeCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/InvokeCommand.java
new file mode 100644
index 0000000..71ef3f0
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/InvokeCommand.java
@@ -0,0 +1,75 @@
+/*
+ * 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.common.command;
+
+import java.util.List;
+import org.apache.ignite.raft.client.WriteCommand;
+
+/**
+ * Represents invoke command for meta storage.
+ */
+public class InvokeCommand implements WriteCommand {
+    /** Condition. */
+    private final ConditionInfo cond;
+
+    /** Success operations. */
+    private final List<OperationInfo> success;
+
+    /** Failure operations. */
+    private final List<OperationInfo> failure;
+
+    /**
+     * Constructs invoke command instance.
+     *
+     * @param cond Condition.
+     * @param success Success operations.
+     * @param failure Failure operations.
+     */
+    public InvokeCommand(ConditionInfo cond, List<OperationInfo> success, List<OperationInfo> failure) {
+        this.cond = cond;
+        this.success = success;
+        this.failure = failure;
+    }
+
+    /**
+     * Returns condition.
+     *
+     * @return Condition.
+     */
+    public ConditionInfo condition() {
+        return cond;
+    }
+
+    /**
+     * Returns success operations.
+     *
+     * @return Success operations.
+     */
+    public List<OperationInfo> success() {
+        return success;
+    }
+
+    /**
+     * Returns failure operations.
+     *
+     * @return Failure operations.
+     */
+    public List<OperationInfo> failure() {
+        return failure;
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/MultipleEntryResponse.java
similarity index 57%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
copy to modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/MultipleEntryResponse.java
index b1fe4cb..a11a3e2 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/MultipleEntryResponse.java
@@ -17,29 +17,31 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
-import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
+import java.io.Serializable;
+import java.util.List;
 
 /**
- * Remove command for {@link MetaStorageCommandListener} that removes an entry for the given key.
+ * Defines response for command which returns a number of results.
  */
-public final class RemoveCommand implements WriteCommand {
-    /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+public class MultipleEntryResponse implements Serializable {
+    /** Single responses. */
+    private final List<SingleEntryResponse> entries;
 
     /**
-     * @param key he key. Couldn't be {@code null}.
+     * Constructs multiple entries response.
+     *
+     * @param entries The ;list of single responses.
      */
-    public RemoveCommand(@NotNull Key key) {
-        this.key = key;
+    public MultipleEntryResponse(List<SingleEntryResponse> entries) {
+        this.entries = entries;
     }
 
     /**
-     * @return The key. Couldn't be {@code null}.
+     * Returns the list of single responses.
+     *
+     * @return The list of single responses.
      */
-    public @NotNull Key key() {
-        return key;
+    public List<SingleEntryResponse> entries() {
+        return entries;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/OperationInfo.java
similarity index 51%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java
copy to modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/OperationInfo.java
index 16205d8..21776a2 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/OperationInfo.java
@@ -17,42 +17,60 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
-import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.metastorage.common.OperationType;
+
+import java.io.Serializable;
 
 /**
- * Put command for {@link MetaStorageCommandListener} that inserts or updates an entry
- * with the given key and the given value and retrieves a previous entry for the given key.
+ * Defines operation.
  */
-public final class PutCommand implements WriteCommand {
-    /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+public class OperationInfo implements Serializable {
+    /** Key. */
+    private final byte[] key;
+
+    /** Value. */
+    private final byte[] val;
 
-    /** The value. Couldn't be {@code null}. */
-    @NotNull private final byte[] val;
+    /** Operation type. */
+    private final OperationType type;
 
     /**
-     * @param key The key. Couldn't be {@code null}.
-     * @param val The value. Couldn't be {@code null}.
+     * Constructs operation with given parameters.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @param type Operation type.
      */
-    public PutCommand(@NotNull Key key, @NotNull byte[] val) {
+    public OperationInfo(byte[] key, byte[] val, OperationType type) {
         this.key = key;
         this.val = val;
+        this.type = type;
+    }
+
+    /**
+     * Returns operation type.
+     *
+     * @return Operation type.
+     */
+    public OperationType type() {
+        return type;
     }
 
     /**
-     * @return The key. Couldn't be {@code null}.
+     * Returns key.
+     *
+     * @return Key.
      */
-    public @NotNull Key key() {
+    public byte[] key() {
         return key;
     }
 
     /**
-     * @return The value. Couldn't be {@code null}.
+     * Returns value.
+     *
+     * @return Value.
      */
-    public @NotNull byte[] value() {
+    public byte[] value() {
         return val;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutAllCommand.java
index 0e87623..fe25f90 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutAllCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutAllCommand.java
@@ -17,38 +17,61 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
-import java.util.HashMap;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Put all command for {@link MetaStorageCommandListener} that inserts or updates entries
+ * Put all command for MetaStorageCommandListener that inserts or updates entries
  * with given keys and given values.
  */
 public final class PutAllCommand implements WriteCommand {
-    /** The map of keys and corresponding values. Couldn't be {@code null} or empty. */
-    @NotNull private final Map<Key, byte[]> vals;
+    /** List of keys. */
+    private final List<byte[]> keys;
+
+    /** List of values. */
+    private final List<byte[]> vals;
 
     /**
      * @param vals he map of keys and corresponding values. Couldn't be {@code null} or empty.
      */
-    public PutAllCommand(@NotNull Map<Key, byte[]> vals) {
+    public PutAllCommand(@NotNull Map<ByteArray, byte[]> vals) {
         assert !vals.isEmpty();
 
-        if (vals instanceof Serializable)
-            this.vals = vals;
-        else
-            this.vals = new HashMap<>(vals);
+        int size = vals.size();
+
+        this.keys = new ArrayList<>(size);
+
+        this.vals = new ArrayList<>(size);
+
+        for (Map.Entry<ByteArray, byte[]> e : vals.entrySet()) {
+            byte[] key = e.getKey().bytes();
+
+            byte[] val = e.getValue();
+
+            assert key != null : "Key could not be null.";
+            assert val != null : "Value could not be null.";
+
+            this.keys.add(key);
+
+            this.vals.add(val);
+        }
+    }
+
+    /**
+     * @return Entries values.
+     */
+    public @NotNull List<byte[]> keys() {
+        return keys;
     }
 
     /**
-     * @return The map of keys and corresponding values. Couldn't be  or empty.
+     * @return Entries values.
      */
-    public @NotNull Map<Key, byte[]> values() {
+    public @NotNull List<byte[]> values() {
         return vals;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java
index 16205d8..fe24ffe 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutCommand.java
@@ -17,18 +17,17 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Put command for {@link MetaStorageCommandListener} that inserts or updates an entry
+ * Put command for MetaStorageCommandListener that inserts or updates an entry
  * with the given key and the given value and retrieves a previous entry for the given key.
  */
 public final class PutCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+    @NotNull private final byte[] key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,15 +36,15 @@ public final class PutCommand implements WriteCommand {
      * @param key The key. Couldn't be {@code null}.
      * @param val The value. Couldn't be {@code null}.
      */
-    public PutCommand(@NotNull Key key, @NotNull byte[] val) {
-        this.key = key;
+    public PutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
+        this.key = key.bytes();
         this.val = val;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull byte[] key() {
         return key;
     }
 
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RangeCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RangeCommand.java
index 9027512..b531454 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RangeCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RangeCommand.java
@@ -17,31 +17,30 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Range command for {@link MetaStorageCommandListener} that retrieves entries for the given
+ * Range command for MetaStorageCommandListener that retrieves entries for the given
  * key range in lexicographic order. Entries will be filtered out by upper bound of given revision number.
  */
 public final class RangeCommand implements WriteCommand {
     /** Start key of range (inclusive). Couldn't be {@code null}. */
-    @NotNull private final Key keyFrom;
+    @NotNull private final byte[] keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final Key keyTo;
+    @Nullable private final byte[] keyTo;
 
     /** The upper bound for entry revision. {@code -1} means latest revision. */
-    @NotNull private final Long revUpperBound;
+    @NotNull private final long revUpperBound;
 
     /**
      * @param keyFrom Start key of range (inclusive).
      * @param keyTo End key of range (exclusive).
      */
-    public RangeCommand(@NotNull Key keyFrom, @Nullable Key keyTo) {
+    public RangeCommand(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo) {
         this(keyFrom, keyTo, -1L);
     }
 
@@ -51,33 +50,33 @@ public final class RangeCommand implements WriteCommand {
      * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
      */
     public RangeCommand(
-        @NotNull Key keyFrom,
-        @Nullable Key keyTo,
-        @NotNull Long revUpperBound
+        @NotNull ByteArray keyFrom,
+        @Nullable ByteArray keyTo,
+        long revUpperBound
     ) {
-        this.keyFrom = keyFrom;
-        this.keyTo = keyTo;
+        this.keyFrom = keyFrom.bytes();
+        this.keyTo = keyTo == null ? null : keyTo.bytes();
         this.revUpperBound = revUpperBound;
     }
 
     /**
      * @return Start key of range (inclusive). Couldn't be {@code null}.
      */
-    public @NotNull Key keyFrom() {
+    public @NotNull byte[] keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable Key keyTo() {
+    public @Nullable byte[] keyTo() {
         return keyTo;
     }
 
     /**
      * @return The upper bound for entry revision. Means latest revision.
      */
-    public @NotNull Long revUpperBound() {
+    public @NotNull long revUpperBound() {
         return revUpperBound;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveAllCommand.java
index 437c368..b40854b 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveAllCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveAllCommand.java
@@ -17,35 +17,34 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Remove all command for {@link MetaStorageCommandListener} that removes entries for given keys.
+ * Remove all command for MetaStorageCommandListener that removes entries for given keys.
  */
 public final class RemoveAllCommand implements WriteCommand {
-    /** The keys collection. Couldn't be {@code null}. */
-    @NotNull private final Collection<Key> keys;
+    /** The keys list. Couldn't be {@code null}. */
+    @NotNull private final List<byte[]> keys;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      */
-    public RemoveAllCommand(@NotNull Collection<Key> keys) {
-        if (keys instanceof Serializable)
-            this.keys = keys;
-        else
-            this.keys = new ArrayList<>(keys);
+    public RemoveAllCommand(@NotNull Set<ByteArray> keys) {
+        this.keys = new ArrayList<>(keys.size());
+
+        for (ByteArray key : keys)
+            this.keys.add(key.bytes());
     }
 
     /**
-     * @return The keys collection. Couldn't be {@code null}.
+     * @return The keys list. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull List<byte[]> keys() {
         return keys;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
index b1fe4cb..b98b829 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
@@ -17,29 +17,28 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Remove command for {@link MetaStorageCommandListener} that removes an entry for the given key.
+ * Remove command for MetaStorageCommandListener that removes an entry for the given key.
  */
 public final class RemoveCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+    @NotNull private final byte[] key;
 
     /**
      * @param key he key. Couldn't be {@code null}.
      */
-    public RemoveCommand(@NotNull Key key) {
-        this.key = key;
+    public RemoveCommand(@NotNull ByteArray key) {
+        this.key = key.bytes();
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull byte[] key() {
         return key;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/SingleEntryResponse.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/SingleEntryResponse.java
new file mode 100644
index 0000000..678ec1a
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/SingleEntryResponse.java
@@ -0,0 +1,92 @@
+/*
+ * 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.common.command;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines response for command which returns exactly one result (entry).
+ */
+public class SingleEntryResponse implements Serializable {
+    /** Key. */
+    @NotNull
+    private final byte[] key;
+
+    /** Value. */
+    @Nullable
+    private final byte[] val;
+
+    /** Revision. */
+    private final long rev;
+
+    /** Update counter */
+    private final long updCntr;
+
+    /**
+     * Constructs single entry response.
+     *
+     * @param key Key. Couldn't be {@code null}.
+     * @param val Value. Could be {@code null} for empty and tombstone entries.
+     * @param rev Revision number.
+     * @param updCntr Update counter.
+     */
+    public SingleEntryResponse(byte[] key, byte[] val, long rev, long updCntr) {
+        this.key = key;
+        this.val = val;
+        this.rev = rev;
+        this.updCntr = updCntr;
+    }
+
+    /**
+     * Returns key.
+     *
+     * @return Entry key. Couldn't be {@code null}.
+     */
+    @NotNull public byte[] key() {
+        return key;
+    }
+
+    /**
+     * Returns value.
+     *
+     * @return Entry value. Could be {@code null} for empty and tombstone entries.
+     */
+    @Nullable public byte[] value() {
+        return val;
+    }
+
+    /**
+     * Returns revision.
+     *
+     * @return Entry revision.
+     */
+    public long revision() {
+        return rev;
+    }
+
+    /**
+     * Returns update counter.
+     *
+     * @return Entry update counter.
+     */
+    public long updateCounter() {
+        return updCntr;
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchExactKeysCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchExactKeysCommand.java
index 0bb9080..87635a3 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchExactKeysCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchExactKeysCommand.java
@@ -17,41 +17,40 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Watch command for {@link MetaStorageCommandListener} that subscribes on meta storage updates matching the parameters.
+ * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
  */
 public final class WatchExactKeysCommand implements WriteCommand {
-    /** The keys collection. Couldn't be {@code null}. */
-    @NotNull private final Collection<Key> keys;
+    /** The keys list. Couldn't be {@code null}. */
+    @NotNull private final List<byte[]> keys;
 
     /** Start revision inclusive. {@code 0} - all revisions. */
-    @NotNull private final Long revision;
+    private final long revision;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      * @param revision Start revision inclusive. {@code 0} - all revisions.
      */
-    public WatchExactKeysCommand(@NotNull Collection<Key> keys, @NotNull Long revision) {
-        if (keys instanceof Serializable)
-            this.keys = keys;
-        else
-            this.keys = new ArrayList<>(keys);
+    public WatchExactKeysCommand(@NotNull Set<ByteArray> keys, long revision) {
+        this.keys = new ArrayList<>(keys.size());
+
+        for (ByteArray key : keys)
+            this.keys.add(key.bytes());
 
         this.revision = revision;
     }
 
     /**
-     * @return The keys collection. Couldn't be {@code null}.
+     * @return The keys list. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull List<byte[]> keys() {
         return keys;
     }
 
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchRangeKeysCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchRangeKeysCommand.java
index f959039..49839cf 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchRangeKeysCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/WatchRangeKeysCommand.java
@@ -17,30 +17,28 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Watch command for {@link MetaStorageCommandListener} that subscribes on meta storage updates matching the parameters.
+ * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
  */
 public final class WatchRangeKeysCommand implements WriteCommand {
     /** Start key of range (inclusive). Couldn't be {@code null}. */
-    @Nullable private final Key keyFrom;
+    @Nullable private final byte[] keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final Key keyTo;
+    @Nullable private final byte[] keyTo;
 
     /** Start revision inclusive. {@code 0} - all revisions. */
-    @NotNull private final Long revision;
+    private final long revision;
 
     /**
      * @param keyFrom Start key of range (inclusive).
      * @param keyTo End key of range (exclusive).
      */
-    public WatchRangeKeysCommand(@Nullable Key keyFrom, @Nullable Key keyTo) {
+    public WatchRangeKeysCommand(@Nullable ByteArray keyFrom, @Nullable ByteArray keyTo) {
         this(keyFrom, keyTo, 0L);
     }
 
@@ -50,33 +48,33 @@ public final class WatchRangeKeysCommand implements WriteCommand {
      * @param revision Start revision inclusive. {@code 0} - all revisions.
      */
     public WatchRangeKeysCommand(
-        @Nullable Key keyFrom,
-        @Nullable Key keyTo,
-        @NotNull Long revision
+        @Nullable ByteArray keyFrom,
+        @Nullable ByteArray keyTo,
+        long revision
     ) {
-        this.keyFrom = keyFrom;
-        this.keyTo = keyTo;
+        this.keyFrom = keyFrom == null ? null : keyFrom.bytes();
+        this.keyTo = keyTo == null ? null : keyTo.bytes();
         this.revision = revision;
     }
 
     /**
      * @return Start key of range (inclusive). Couldn't be {@code null}.
      */
-    public @Nullable Key keyFrom() {
+    public @Nullable byte[] keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable Key keyTo() {
+    public @Nullable byte[] keyTo() {
         return keyTo;
     }
 
     /**
      * @return Start revision inclusive. {@code 0} - all revisions.
      */
-    public @NotNull Long revision() {
+    public long revision() {
         return revision;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorCloseCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorCloseCommand.java
index d07b9a8..aae9c32 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorCloseCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorCloseCommand.java
@@ -18,12 +18,11 @@
 package org.apache.ignite.internal.metastorage.common.command.cursor;
 
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Cursor close command for {@link MetaStorageCommandListener} that closes cursor with given id.
+ * Cursor close command for MetaStorageCommandListener that closes cursor with given id.
  */
 public class CursorCloseCommand implements WriteCommand {
     /** Cursor id. */
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorHasNextCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorHasNextCommand.java
index 142369a..461c2ce 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorHasNextCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorHasNextCommand.java
@@ -18,12 +18,11 @@
 package org.apache.ignite.internal.metastorage.common.command.cursor;
 
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Cursor {@code hasNext} command for {@link MetaStorageCommandListener} that checks whether next element is available.
+ * Cursor {@code hasNext} command for MetaStorageCommandListener that checks whether next element is available.
  */
 public class CursorHasNextCommand implements ReadCommand {
     /** Cursor id. */
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorNextCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorNextCommand.java
index e9ba6a7..78b36a6 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorNextCommand.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/cursor/CursorNextCommand.java
@@ -18,12 +18,11 @@
 package org.apache.ignite.internal.metastorage.common.command.cursor;
 
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Cursor {@code next} command for {@link MetaStorageCommandListener} that returns next element and moves cursor.
+ * Cursor {@code next} command for MetaStorageCommandListener that returns next element and moves cursor.
  */
 public class CursorNextCommand implements WriteCommand {
     /** Cursor id. */
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java b/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java
deleted file mode 100644
index 54049de..0000000
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java
+++ /dev/null
@@ -1,370 +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.metastorage.common;
-
-import java.util.Arrays;
-
-/**
- * Represents a condition for conditional update.
- */
-public final class Condition {
-    /** Actual condition implementation. */
-    private final InnerCondition cond;
-
-    /**
-     * Constructs a condition which wraps the actual condition implementation.
-     *
-     * @param cond The actual condition implementation.
-     */
-    Condition(InnerCondition cond) {
-        this.cond = cond;
-    }
-
-    /**
-     * Tests the given entry on satisfaction of the condition.
-     *
-     * @param e Entry.
-     * @return The result of condition test. {@code true} - if the entry satisfies to the condition,
-     * otherwise - {@code false}.
-     */
-    public boolean test(Entry e) {
-        return cond.test(e);
-    }
-
-    /**
-     * Represents condition on entry revision. Only one type of condition could be applied to
-     * the one instance of condition. Subsequent invocations of any method which produces condition will throw
-     * {@link IllegalStateException}.
-     */
-    public static final class RevisionCondition implements InnerCondition {
-        /**
-         * The type of condition.
-         *
-         * @see Type
-         */
-        private Type type;
-
-        /** The revision as the condition argument. */
-        private long rev;
-
-        /** Key of entry, which will be tested for condition. */
-        private final Key key;
-
-        /**
-         * Creates a new condition for the given {@code key}.
-         *
-         * @param key Key of entry, to be tested for the condition.
-         */
-        RevisionCondition(Key key) {
-            this.key = key;
-        }
-
-        /**
-         * Produces the condition of type {@link Type#EQUAL}. This condition tests the given revision on equality with
-         * target entry revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition eq(long rev) {
-            validate(type);
-
-            this.type = Type.EQUAL;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#NOT_EQUAL}. This condition tests the given revision on inequality
-         * with target entry revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#NOT_EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition ne(long rev) {
-            validate(type);
-
-            this.type = Type.NOT_EQUAL;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#GREATER}. This condition tests that the target entry revision
-         * is greater than given revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#GREATER}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition gt(long rev) {
-            validate(type);
-
-            this.type = Type.GREATER;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#GREATER_OR_EQUAL}. This condition tests that the target entry
-         * revision is greater than or equal to given revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#GREATER_OR_EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition ge(long rev) {
-            validate(type);
-
-            this.type = Type.GREATER_OR_EQUAL;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#LESS}. This condition tests that target entry revision
-         * is less than the given revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#LESS}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition lt(long rev) {
-            validate(type);
-
-            this.type = Type.LESS;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#LESS_OR_EQUAL}. This condition tests that target entry revision
-         * is less than or equal to the given revision.
-         *
-         * @param rev The revision.
-         * @return The condition of type {@link Type#LESS_OR_EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition le(long rev) {
-            validate(type);
-
-            this.type = Type.LESS_OR_EQUAL;
-            this.rev = rev;
-
-            return new Condition(this);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean test(Entry e) {
-            if ((e.key() == key) || (e.key() != null && e.key().equals(key))) {
-                int res = Long.compare(e.revision(), rev);
-
-                return type.test(res);
-            }
-            else
-                return false;
-        }
-
-        /**
-         * Defines possible condition types which can be applied to the revision.
-         */
-        enum Type {
-            /** Equality condition type. */
-            EQUAL {
-                @Override public boolean test(long res) {
-                    return res == 0;
-                }
-            },
-
-            /** Inequality condition type. */
-            NOT_EQUAL {
-                @Override public boolean test(long res) {
-                    return res != 0;
-                }
-            },
-
-            /** Greater than condition type. */
-            GREATER {
-                @Override public boolean test(long res) {
-                    return res > 0;
-                }
-            },
-
-            /** Less than condition type. */
-            LESS {
-                @Override public boolean test(long res) {
-                    return res < 0;
-                }
-            },
-
-            /** Less than or equal to condition type. */
-            LESS_OR_EQUAL {
-                @Override public boolean test(long res) {
-                    return res <= 0;
-                }
-            },
-
-            /** Greater than or equal to condition type. */
-            GREATER_OR_EQUAL {
-                @Override public boolean test(long res) {
-                    return res >= 0;
-                }
-            };
-
-            /**
-             * Interprets comparison result.
-             *
-             * @param res The result of comparison.
-             * @return The interpretation of the comparison result.
-             */
-            public abstract boolean test(long res);
-        }
-    }
-
-    /**
-     * Represents condition on entry value. Only one type of condition could be applied to
-     * the one instance of condition. Subsequent invocations of any method which produces condition will throw
-     * {@link IllegalStateException}.
-     */
-    public static final class ValueCondition implements InnerCondition {
-        /**
-         * The type of condition.
-         *
-         * @see Type
-         */
-        private Type type;
-
-        /** The value as the condition argument. */
-        private byte[] val;
-
-        /** Key of entry, which will be tested for condition. */
-        private final Key key;
-
-        /**
-         * Creates a new condition for the given {@code key}.
-         *
-         * @param key Key of entry, to be tested for the condition.
-         */
-        ValueCondition(Key key) {
-            this.key = key;
-        }
-
-        /**
-         * Produces the condition of type {@link Type#EQUAL}. This condition tests the given value on equality with
-         * target entry value.
-         *
-         * @param val The value.
-         * @return The condition of type {@link Type#EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition eq(byte[] val) {
-            validate(type);
-
-            this.type = Type.EQUAL;
-            this.val = val;
-
-            return new Condition(this);
-        }
-
-        /**
-         * Produces the condition of type {@link Type#NOT_EQUAL}. This condition tests the given value on inequality
-         * with target entry value.
-         *
-         * @param val The value.
-         * @return The condition of type {@link Type#NOT_EQUAL}.
-         * @throws IllegalStateException In case when the condition is already defined.
-         */
-        public Condition ne(byte[] val) {
-            validate(type);
-
-            this.type = Type.NOT_EQUAL;
-            this.val = val;
-
-            return new Condition(this);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean test(Entry e) {
-            if ((e.key() == key) || (e.key() != null && e.key().equals(key))) {
-                int res = Arrays.compare(e.value(), val);
-
-                return type.test(res);
-            }
-            else
-               return false;
-        }
-
-        /**
-         * Defines possible condition types which can be applied to the value.
-         */
-        enum Type {
-            /** Equality condition type. */
-            EQUAL {
-                @Override public boolean test(long res) {
-                    return res == 0;
-                }
-            },
-
-            /** Inequality condition type. */
-            NOT_EQUAL {
-                @Override public boolean test(long res) {
-                    return res != 0;
-                }
-            };
-
-            /**
-             * Interprets comparison result.
-             *
-             * @param res The result of comparison.
-             * @return The interpretation of the comparison result.
-             */
-            public abstract boolean test(long res);
-        }
-    }
-
-    /**
-     * Checks that condition is not defined yet. If the condition is already defined then exception will be thrown.
-     *
-     * @throws IllegalStateException In case when the condition is already defined.
-     */
-    private static void validate(Enum<?> type) {
-        if (type != null)
-            throw new IllegalStateException("Condition type " + type.name() + " is already defined.");
-    }
-
-    /**
-     * Defines condition interface.
-     */
-    private interface InnerCondition {
-        /**
-         * Tests the given entry on satisfaction of the condition.
-         *
-         * @param e Entry.
-         * @return The result of condition test. {@code true} - if the entry satisfies to the condition,
-         * otherwise - {@code false}.
-         */
-        boolean test(Entry e);
-    }
-}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorage.java b/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorage.java
deleted file mode 100644
index 6155112..0000000
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorage.java
+++ /dev/null
@@ -1,91 +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.metastorage.common;
-
-import java.util.Collection;
-import java.util.List;
-import org.jetbrains.annotations.NotNull;
-
-// TODO: IGNITE-14389 Tmp, used instead of real KeyValueStorage interface from metastorage-server module.
-/**
- *
- */
-@SuppressWarnings("unused") public interface KeyValueStorage {
-    /** */
-    long revision();
-
-    /** */
-    long updateCounter();
-
-    /** */
-    @NotNull Entry get(byte[] key);
-
-    /** */
-    @NotNull Entry get(byte[] key, long rev);
-
-    /** */
-    @NotNull Collection<Entry> getAll(List<byte[]> keys);
-
-    /** */
-    @NotNull Collection<Entry> getAll(List<byte[]> keys, long revUpperBound);
-
-    /** */
-    void put(byte[] key, byte[] value);
-
-    /** */
-    @NotNull Entry getAndPut(byte[] key, byte[] value);
-
-    /** */
-    void putAll(List<byte[]> keys, List<byte[]> values);
-
-    /** */
-    @NotNull Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values);
-
-    /** */
-    void remove(byte[] key);
-
-    /** */
-    @NotNull Entry getAndRemove(byte[] key);
-
-    /** */
-    void removeAll(List<byte[]> key);
-
-    /** */
-    @NotNull Collection<Entry> getAndRemoveAll(List<byte[]> keys);
-
-    /** */
-    boolean invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure);
-
-    /** */
-    Cursor<Entry> range(byte[] keyFrom, byte[] keyTo);
-
-    /** */
-    Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound);
-
-    /** */
-    Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev);
-
-    /** */
-    Cursor<WatchEvent> watch(byte[] key, long rev);
-
-    /** */
-    Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
-
-    /** */
-    void compact();
-}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorageImpl.java b/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorageImpl.java
deleted file mode 100644
index cb59227..0000000
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/KeyValueStorageImpl.java
+++ /dev/null
@@ -1,183 +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.metastorage.common;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.ignite.internal.metastorage.common.DummyEntry;
-import org.jetbrains.annotations.NotNull;
-
-// TODO: IGNITE-14389 Tmp, should be removed.
-/**
- *
- */
-@SuppressWarnings("ConstantConditions")
-public class KeyValueStorageImpl implements KeyValueStorage {
-    /** {@inheritDoc} */
-    @Override public long revision() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long updateCounter() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Entry get(byte[] key) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Entry get(byte[] key, long rev) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Collection<Entry> getAll(List<byte[]> keys) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Collection<Entry> getAll(List<byte[]> keys, long revUpperBound) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void put(byte[] key, byte[] value) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Entry getAndPut(byte[] key, byte[] value) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(List<byte[]> keys, List<byte[]> values) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(byte[] key) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Entry getAndRemove(byte[] key) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(List<byte[]> keys) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public @NotNull Collection<Entry> getAndRemoveAll(List<byte[]> keys) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure) {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
-        return new Cursor<Entry>() {
-            @NotNull @Override public Iterator<Entry> iterator() {
-                return new Iterator<Entry>() {
-                    @Override public boolean hasNext() {
-                        return false;
-                    }
-
-                    @Override public Entry next() {
-                        return null;
-                    }
-                };
-            }
-
-            @Override public void close() throws Exception {
-
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
-        return new Cursor<>() {
-            /** {@inheritDoc} */
-            @Override public void close(){
-
-            }
-
-            /** {@inheritDoc} */
-            @NotNull @Override public Iterator<WatchEvent> iterator() {
-                return new Iterator<>() {
-                    @Override public boolean hasNext() {
-                        return true;
-                    }
-
-                    @Override public WatchEvent next() {
-                        return new WatchEvent(
-                            new DummyEntry(
-                                new Key(new byte[] {1}),
-                                new byte[] {2},
-                                1L,
-                                1L
-                            ),
-                            new DummyEntry(
-                                new Key(new byte[] {1}),
-                                new byte[] {3},
-                                2L,
-                                2L
-                            )
-                        );
-                    }
-                };
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public Cursor<WatchEvent> watch(byte[] key, long rev) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void compact() {
-
-    }
-}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java b/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java
deleted file mode 100644
index 9810fe5..0000000
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java
+++ /dev/null
@@ -1,93 +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.metastorage.common;
-
-/**
- * Defines operation for meta storage conditional update (invoke).
- */
-public final class Operation {
-    /** Actual operation implementation. */
-    @SuppressWarnings({"FieldCanBeLocal", "unused"})
-    private final InnerOp upd;
-
-    /**
-     * Constructs an operation which wraps the actual operation implementation.
-     *
-     * @param upd The actual operation implementation.
-     */
-    Operation(InnerOp upd) {
-        this.upd = upd;
-    }
-
-    /**
-     * Represents operation of type <i>remove</i>.
-     */
-    public static final class RemoveOp implements InnerOp {
-        /** Key. */
-        private final Key key;
-
-        /**
-         * Creates a new remove operation for the given {@code key}.
-         *
-         * @param key Key.
-         */
-        RemoveOp(Key key) {
-            this.key = key;
-        }
-    }
-
-    /**
-     * Represents operation of type <i>put</i>.
-     */
-    public static final class PutOp implements InnerOp {
-        /** Key. */
-        private final Key key;
-
-        /** Value. */
-        private final byte[] val;
-
-        /**
-         * Constructs operation of type <i>put</i>.
-         *
-         * @param val The value to which the entry should be updated.
-         */
-        PutOp(Key key, byte[] val) {
-            this.key = key;
-            this.val = val;
-        }
-    }
-
-    /**
-     * Represents operation of type <i>no-op</i>.
-     */
-    public static final class NoOp implements InnerOp {
-        /**
-         * Default no-op constructor.
-         */
-        NoOp() {
-            // No-op.
-        }
-    }
-
-    /**
-     * Defines operation interface.
-     */
-    private interface InnerOp {
-        // Marker interface.
-    }
-}
diff --git a/modules/metastorage-server/pom.xml b/modules/metastorage-server/pom.xml
new file mode 100644
index 0000000..5f0f453
--- /dev/null
+++ b/modules/metastorage-server/pom.xml
@@ -0,0 +1,64 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent/pom.xml</relativePath>
+    </parent>
+
+    <artifactId>ignite-metastorage-server</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-metastorage-common</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.jetbrains</groupId>
+            <artifactId>annotations</artifactId>
+        </dependency>
+
+        <!-- Test dependencies. -->
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-engine</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/AbstractCondition.java
similarity index 58%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
copy to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/AbstractCondition.java
index b1fe4cb..df4acbe 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/RemoveCommand.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/AbstractCondition.java
@@ -15,31 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.metastorage.common.command;
+package org.apache.ignite.internal.metastorage.server;
 
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.raft.MetaStorageCommandListener;
-import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * Remove command for {@link MetaStorageCommandListener} that removes an entry for the given key.
+ * An abstract condition which could be applied to an entry identified by the key.
  */
-public final class RemoveCommand implements WriteCommand {
-    /** The key. Couldn't be {@code null}. */
-    @NotNull private final Key key;
+public abstract class AbstractCondition implements Condition {
+    /** Entry key. */
+    @NotNull
+    private final byte[] key;
 
     /**
-     * @param key he key. Couldn't be {@code null}.
+     * Constructs a condition with the given entry key.
+     *
+     * @param key Key identifies an entry which the condition will applied to.
      */
-    public RemoveCommand(@NotNull Key key) {
+    public AbstractCondition(@NotNull byte[] key) {
         this.key = key;
     }
 
-    /**
-     * @return The key. Couldn't be {@code null}.
-     */
-    public @NotNull Key key() {
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public byte[] key() {
         return key;
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
similarity index 97%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java
rename to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
index 6ea9f43..85cbf3d 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.metastorage.server;
 
 /**
  * Thrown when a requested operation on meta storage could not be performed because target revisions were removed
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
similarity index 55%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
copy to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
index 1ed196e..e20459a 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
@@ -15,28 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.metastorage.server;
 
 import org.jetbrains.annotations.NotNull;
 
+import java.util.Collection;
+
 /**
- * The listener which receives and handles watch updates.
+ * Defines interface for condition which could be applied to an entry.
+ * 
+ * @see KeyValueStorage#invoke(Condition, Collection, Collection) 
  */
-public interface WatchListener {
+public interface Condition {
     /**
-     * The method will be called on each meta storage update.
+     * Returns the key which identifies an entry which condition will applied to.
      *
-     * @param events 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.
+     * @return The key which identifies an entry which condition will applied to.
      */
-    boolean onUpdate(@NotNull Iterable<WatchEvent> events);
+    @NotNull byte[] key();
 
     /**
-     * The method will be called in case of an error occurred. The listener and corresponding watch will be
-     * unregistered.
+     * Tests the given entry on condition.
      *
-     * @param e Exception.
+     * @param e Entry which will be tested on the condition. Can't be {@code null}.
+     * @return {@code True} if the given entry satisfies to the condition, otherwise - {@code false}.
      */
-    void onError(@NotNull Throwable e);
+    boolean test(@NotNull Entry e);
 }
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Entry.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Entry.java
new file mode 100644
index 0000000..0b7670b
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Entry.java
@@ -0,0 +1,190 @@
+/*
+ * 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 org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents a storage unit as entry with key, value and revision, where
+ * <ul>
+ *     <li>key - an unique entry's key represented by an array of bytes. Keys are comparable in lexicographic manner.</li>
+ *     <li>value - a data which is associated with a key and represented as an array of bytes.</li>
+ *     <li>revision - a number which denotes a version of whole meta storage.
+ *     Each change (which could include multiple entries) increments the revision. </li>
+ *     <li>updateCounter - a number which increments on every update in the change under one revision.</li>
+ * </ul>
+ *
+ * Instance of {@link #Entry} could represents:
+ * <ul>
+ *     <li>A regular entry which stores a particular key, a value and a revision number.</li>
+ *     <li>An empty entry which denotes absence a regular entry in the meta storage for a given key.
+ *     A revision is 0 for such kind of entry.</li>
+ *     <li>A tombstone entry which denotes that a regular entry for a given key was removed from storage on some revision.</li>
+ * </ul>
+ */
+public class Entry {
+    /** Entry key. Couldn't be {@code null}. */
+    @NotNull
+    private final byte[] key;
+
+    /**
+     * Entry value.
+     * <p>
+     *     {@code val == null} only for {@link #empty()} and {@link #tombstone()} entries.
+     * </p>
+     */
+    @Nullable
+    private final byte[] val;
+
+    /**
+     * Revision number corresponding to this particular entry.
+     * <p>
+     *     {@code rev == 0} for {@link #empty()} entry,
+     *     {@code rev > 0} for regular and {@link #tombstone()} entries.
+     * </p>
+     */
+    private final long rev;
+
+    /**
+     * Update counter corresponds to this particular entry.
+     * <p>
+     *     {@code updCntr == 0} for {@link #empty()} entry,
+     *     {@code updCntr > 0} for regular and {@link #tombstone()} entries.
+     * </p>
+     */
+    private final long updCntr;
+
+    /**
+     * Constructor.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @param val Value bytes. Couldn't be {@code null}.
+     * @param rev Revision.
+     * @param updCntr Update counter.
+     */
+    // TODO: It seems user will never create Entry, so we can reduce constructor scope to protected or package-private and reuse it from two-place private constructor.
+    public Entry(@NotNull byte[] key, @NotNull byte[] val, long rev, long updCntr) {
+        assert key != null : "key can't be null";
+        assert val != null : "value can't be null";
+
+        this.key = key;
+        this.val = val;
+        this.rev = rev;
+        this.updCntr = updCntr;
+    }
+
+    /**
+     * Constructor for empty and tombstone entries.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @param rev Revision.
+     * @param updCntr Update counter.
+     */
+    private Entry(@NotNull byte[] key, long rev, long updCntr) {
+        assert key != null : "key can't be null";
+
+        this.key = key;
+        this.val = null;
+        this.rev = rev;
+        this.updCntr = updCntr;
+    }
+
+    /**
+     * Creates an instance of empty entry for a given key.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @return Empty entry.
+     */
+    @NotNull
+    public static Entry empty(byte[] key) {
+        return new Entry(key, 0, 0);
+    }
+
+    /**
+     * Creates an instance of tombstone entry for a given key and a revision.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @param rev Revision.
+     * @param updCntr Update counter.
+     * @return Empty entry.
+     */
+    @NotNull
+    public static Entry tombstone(byte[] key, long rev, long updCntr) {
+        assert rev > 0 : "rev must be positive for tombstone entry.";
+        assert updCntr > 0 : "updCntr must be positive for tombstone entry.";
+
+        return new Entry(key, rev, updCntr);
+    }
+
+    /**
+     * Returns a key.
+     *
+     * @return Key.
+     */
+    @NotNull
+    public byte[] key() {
+        return key;
+    }
+
+    /**
+     * Returns a value.
+     *
+     * @return Value.
+     */
+    @Nullable
+    public byte[] value() {
+        return val;
+    }
+
+    /**
+     * Returns a revision.
+     *
+     * @return Revision.
+     */
+    public long revision() {
+        return rev;
+    }
+
+    /**
+     * Returns a update counter.
+     *
+     * @return Update counter.
+     */
+    public long updateCounter() {
+        return updCntr;
+    }
+
+    /**
+     * Returns value which denotes whether entry is tombstone or not.
+     *
+     * @return {@code True} if entry is tombstone, otherwise - {@code false}.
+     */
+    public boolean tombstone() {
+        return val == null && rev > 0 && updCntr > 0;
+    }
+
+    /**
+     * Returns value which denotes whether entry is empty or not.
+     *
+     * @return {@code True} if entry is empty, otherwise - {@code false}.
+     */
+    public boolean empty() {
+        return val == null && rev == 0 && updCntr == 0;
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
similarity index 52%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
rename to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
index 1ed196e..ef0adba 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
@@ -15,28 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
-
-import org.jetbrains.annotations.NotNull;
+package org.apache.ignite.internal.metastorage.server;
 
 /**
- * The listener which receives and handles watch updates.
+ * Represent an update event for particular key and entry.
  */
-public interface WatchListener {
+public class EntryEvent {
+    /** Old (previous) entry. */
+    private final Entry oldEntry;
+
+    /** New (current) entry. */
+    private final Entry entry;
+
+    /**
+     * Constructs event with given old and new entries.
+     *
+     * @param oldEntry Old entry.
+     * @param curEntry New entry.
+     */
+    public EntryEvent(Entry oldEntry, Entry curEntry) {
+        this.oldEntry = oldEntry;
+        this.entry = curEntry;
+    }
+
     /**
-     * The method will be called on each meta storage update.
+     * Returns old entry.
      *
-     * @param events 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.
+     * @return Old entry.
      */
-    boolean onUpdate(@NotNull Iterable<WatchEvent> events);
+    public Entry oldEntry() {
+        return oldEntry;
+    }
 
     /**
-     * The method will be called in case of an error occurred. The listener and corresponding watch will be
-     * unregistered.
+     * Rreturns new entry.
      *
-     * @param e Exception.
+     * @return New entry.
      */
-    void onError(@NotNull Throwable e);
+    public Entry entry() {
+        return entry;
+    }
 }
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ExistenceCondition.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ExistenceCondition.java
new file mode 100644
index 0000000..506e23b
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ExistenceCondition.java
@@ -0,0 +1,74 @@
+/*
+ * 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 org.jetbrains.annotations.NotNull;
+
+/**
+ * Condition tests an entry on existence in meta storage.
+ * Entry exists if it is not empty and not tombstone.
+ */
+public class ExistenceCondition extends AbstractCondition {
+    /** Condition type. */
+    @NotNull
+    private final Type type;
+
+    /**
+     * Constructs existence condition with the given type and for entry identified by the given key.
+     *
+     * @param type Condition type. Can't be {@code null}.
+     * @param key Key of entry which condition will be applied to. Can't be {@code null}.
+     */
+    public ExistenceCondition(@NotNull Type type, @NotNull byte[] key) {
+        super(key);
+
+        this.type = type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(@NotNull Entry e) {
+        boolean res = !(e.empty() || e.tombstone());
+
+        return type.test(res);
+    }
+
+    /** Defines existence condition types. */
+    public enum Type {
+        /** Equality condition type. */
+        EXISTS {
+            @Override public boolean test(boolean res) {
+                return res;
+            }
+        },
+
+        /** Inequality condition type. */
+        NOT_EXISTS {
+            @Override public boolean test(boolean res) {
+                return !res;
+            }
+        };
+
+        /**
+         * Interprets comparison result.
+         *
+         * @param res The result of comparison.
+         * @return The interpretation of the comparison result.
+         */
+        public abstract boolean test(boolean res);
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
new file mode 100644
index 0000000..d4c7da5
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
@@ -0,0 +1,206 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Defines key/value storage interface.
+ */
+public interface KeyValueStorage {
+    /**
+     * Returns storage revision.
+     *
+     * @return Storage revision.
+     */
+    long revision();
+
+    /**
+     * Returns update counter.
+     *
+     * @return Update counter.
+     */
+    long updateCounter();
+
+    /**
+     * Returns an entry by the given key.
+     *
+     * @param key The key.
+     * @return Value corresponding to the given key.
+     */
+    @NotNull Entry get(byte[] key);
+
+    /**
+     * Returns an entry by the given key and revision.
+     *
+     * @param key The key.
+     * @param rev The revision.
+     * @return Value corresponding to the given key.
+     */
+    @NotNull Entry get(byte[] key, long rev);
+
+    /**
+     * Returns all entries corresponding to given keys.
+     *
+     * @param keys Keys collection.
+     * @return Entries corresponding to given keys.
+     */
+    @NotNull Collection<Entry> getAll(List<byte[]> keys);
+
+    /**
+     * Returns all entries corresponding to given keys and bounded by the given revision.
+     *
+     * @param keys Keys collection.
+     * @param revUpperBound Upper bound of revision.
+     * @return Entries corresponding to given keys.
+     */
+    @NotNull Collection<Entry> getAll(List<byte[]> keys, long revUpperBound);
+
+    /**
+     * Inserts an entry with the given key and given value.
+     *
+     * @param key The key.
+     * @param value The value.
+     */
+    void put(@NotNull byte[] key, @NotNull byte[] value);
+
+    /**
+     * Inserts an entry with the given key and given value and returns previous entry.
+     *
+     * @param key The key.
+     * @param value The value.
+     * @return Previous entry corresponding to the given key.
+     */
+    @NotNull Entry getAndPut(byte[] key, byte[] value);
+
+    /**
+     * Inserts entries with given keys and given values.
+     *
+     * @param keys The key list.
+     * @param values The values list.
+     */
+    void putAll(List<byte[]> keys, List<byte[]> values);
+
+    /**
+     * Inserts entries with given keys and given values and returns previous entries.
+     *
+     * @param keys The key list.
+     * @param values The values list.
+     * @return Collection of previous entries corresponding to given keys.
+     */
+    @NotNull Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values);
+
+    /**
+     * Removes an entry with the given key.
+     *
+     * @param key The key.
+     */
+    void remove(byte[] key);
+
+    /**
+     * Removes an entry with the given key and returns previous entry.
+     *
+     * @param key The key.
+     * @return Previous entry.
+     */
+    @NotNull Entry getAndRemove(byte[] key);
+
+    /**
+     * Remove all entries corresponding to given keys.
+     *
+     * @param keys The keys list.
+     */
+    void removeAll(List<byte[]> keys);
+
+    /**
+     * Remove all entries corresponding to given keys and returns previous entries.
+     *
+     * @param keys The keys list.
+     * @return Previous entries.
+     */
+    @NotNull Collection<Entry> getAndRemoveAll(List<byte[]> keys);
+
+    /**
+     * Performs {@code success} operation if condition is {@code true}, otherwise performs
+     * {@code failure} operations.
+     *
+     * @param condition Condition.
+     * @param success Success operations.
+     * @param failure Failure operations.
+     * @return Result of test condition.
+     */
+    boolean invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure);
+
+    /**
+     * Returns cursor by entries which correspond to the given keys range.
+     *
+     * @param keyFrom Start key of range (inclusive).
+     * @param keyTo Last key of range (exclusive).
+     * @return Cursor by entries which correspond to the given keys range.
+     */
+    Cursor<Entry> range(byte[] keyFrom, byte[] keyTo);
+
+    /**
+     * Returns cursor by entries which correspond to the given keys range and bounded by revision number..
+     *
+     * @param keyFrom Start key of range (inclusive).
+     * @param keyTo Last key of range (exclusive).
+     * @param revUpperBound Upper bound of revision.
+     * @return Cursor by entries which correspond to the given keys range.
+     */
+    Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound);
+
+    /**
+     * Creates subscription on updates of entries corresponding to the given keys range and starting from
+     * the given revision number.
+     *
+     * @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[] keyTo, long rev);
+
+    /**
+     * Creates subscription on updates of entries corresponding to the given keys range (where upper bound is unlimited)
+     * and starting from the given revision number.
+     *
+     * @param key Start key of range (inclusive).
+     * @param rev Start revision number.
+     * @return Cursor by update events.
+     */
+    Cursor<WatchEvent> watch(byte[] key, long rev);
+
+    /**
+     * Creates subscription on updates of entries corresponding to the given keys collection
+     * and starting from the given revision number.
+     *
+     * @param keys Collection of keys
+     * @param rev Start revision number.
+     * @return Cursor by update events.
+     */
+    Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
+
+    /**
+     * Compacts storage (removes tombstones).
+     */
+    void compact();
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Operation.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Operation.java
new file mode 100644
index 0000000..4c69d7d
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Operation.java
@@ -0,0 +1,100 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines operation which will be applied to an entry identified by the key.
+ * <p>
+ * Invariants:
+ * <ul>
+ *     <li>Any operation identifies a target entry by not null {@code key} except of {@link OperationType#NO_OP}.</li>
+ *     <li>Only {@link OperationType#PUT} operation contains value which will be written to meta storage.</li>
+ * </ul>
+ */
+public final class Operation {
+    /**
+     * Key identifies an entry which operation will be applied to. Key is {@code null} for {@link OperationType#NO_OP} operation.
+     */
+    @Nullable
+    private final byte[] key;
+
+    /**
+     * Value which will be associated with the {@link #key}. Value is not {@code null} only for {@link OperationType#PUT}
+     * operation.
+     */
+    @Nullable
+    private final byte[] val;
+
+    /**
+     * Operation type.
+     * @see OperationType
+     */
+    @NotNull
+    private final OperationType type;
+
+    /**
+     * Constructs operation which will be applied to an entry identified by the given key.
+     *
+     * @param type Operation type. Can't be {@code null}.
+     * @param key Key identifies an entry which operation will be applied to.
+     * @param val Value will be associated with an entry identified by the {@code key}.
+     */
+    public Operation(@NotNull OperationType type, @Nullable byte[] key, @Nullable byte[] val) {
+        assert (type == OperationType.NO_OP && key == null && val == null)
+                || (type == OperationType.PUT && key != null && val != null)
+                || (type == OperationType.REMOVE && key != null && val == null)
+                : "Invalid operation parameters: [type=" + type + ", key=" + Objects.toString(key,"null") +
+                ", val=" + Objects.toString(key,"null") + ']';
+
+        this.key = key;
+        this.val = val;
+        this.type = type;
+    }
+
+    /**
+     * Returns a key which identifies an entry which operation will be applied to.
+     *
+     * @return A key which identifies an entry which operation will be applied to.
+     */
+    @Nullable public byte[] key() {
+        return key;
+    }
+
+    /**
+     * Returns a value which will be associated with an entry identified by the {@code key}.
+     *
+     * @return A value which will be associated with an entry identified by the {@code key}.
+     */
+    @Nullable public byte[] value() {
+        return val;
+    }
+
+    /**
+     * Returns an operation type.
+     *
+     * @return An operation type.
+     */
+    @NotNull public OperationType type() {
+        return type;
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/RevisionCondition.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/RevisionCondition.java
new file mode 100644
index 0000000..94e8b71
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/RevisionCondition.java
@@ -0,0 +1,110 @@
+/*
+ * 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 org.jetbrains.annotations.NotNull;
+
+/**
+ * Defines condition on entry revision.
+ */
+public class RevisionCondition extends AbstractCondition {
+    /** Condition type. */
+    @NotNull
+    private final Type type;
+
+    /** Revision which will be tested against an entry revision. Must be positive. */
+    private final long rev;
+
+    /**
+     * Constructs revision condition with the given type, the key and revision.
+     *
+     * @param type Condition type. Can't be {@code null}.
+     * @param key Key identifies an entry which condition will be applied to. Can't be {@code null}.
+     * @param rev Revision which will be tested against an entry revision. Must be positive.
+     */
+    public RevisionCondition(@NotNull Type type, @NotNull byte[] key, long rev) {
+        super(key);
+
+        assert rev > 0 : "Revision must be positive.";
+
+        this.type = type;
+        this.rev = rev;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(@NotNull Entry e) {
+        int res = Long.compare(e.revision(), rev);
+
+        return type.test(res);
+    }
+
+    /**
+     * Defines possible condition types which can be applied to the revision.
+     */
+    public enum Type {
+        /** Equality condition type. */
+        EQUAL {
+            @Override public boolean test(long res) {
+                return res == 0;
+            }
+        },
+
+        /** Inequality condition type. */
+        NOT_EQUAL {
+            @Override public boolean test(long res) {
+                return res != 0;
+            }
+        },
+
+        /** Greater than condition type. */
+        GREATER {
+            @Override public boolean test(long res) {
+                return res > 0;
+            }
+        },
+
+        /** Less than condition type. */
+        LESS {
+            @Override public boolean test(long res) {
+                return res < 0;
+            }
+        },
+
+        /** Less than or equal to condition type. */
+        LESS_OR_EQUAL {
+            @Override public boolean test(long res) {
+                return res <= 0;
+            }
+        },
+
+        /** Greater than or equal to condition type. */
+        GREATER_OR_EQUAL {
+            @Override public boolean test(long res) {
+                return res >= 0;
+            }
+        };
+
+        /**
+         * Interprets comparison result.
+         *
+         * @param res The result of comparison.
+         * @return The interpretation of the comparison result.
+         */
+        public abstract boolean test(long res);
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
new file mode 100644
index 0000000..271b8df
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -0,0 +1,775 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NoSuchElementException;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.metastorage.server.Value.TOMBSTONE;
+
+/**
+ * Simple in-memory key/value storage.
+ *
+ * WARNING: Only for test purposes.
+ */
+public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
+    /** Lexicographical comparator. */
+    private static final Comparator<byte[]> CMP = Arrays::compare;
+
+    /**
+     * Special value for revision number which means that operation should be applied
+     * to the latest revision of an entry.
+     */
+    private static final long LATEST_REV = -1;
+
+    /** Keys index. Value is the list of all revisions under which entry corresponding to the key was modified. */
+    private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(CMP);
+
+    /**  Revisions index. Value contains all entries which were modified under particular revision. */
+    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;
+
+    /** Update counter. Will be incremented for each update of any particular entry. */
+    private long updCntr;
+
+    /** All operations are queued on this lock. */
+    private final Object mux = new Object();
+
+    /** {@inheritDoc} */
+    @Override public long revision() {
+        return rev;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long updateCounter() {
+        return updCntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void put(byte[] key, byte[] value) {
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            doPut(key, value, curRev);
+
+            rev = curRev;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Entry getAndPut(byte[] key, byte[] bytes) {
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            long lastRev = doPut(key, bytes, curRev);
+
+            rev = curRev;
+
+            // Return previous value.
+            return doGetValue(key, lastRev);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void putAll(List<byte[]> keys, List<byte[]> values) {
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            doPutAll(curRev, keys, values);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
+        Collection<Entry> res;
+
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            res = doGetAll(keys, curRev);
+
+            doPutAll(curRev, keys, values);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Entry get(byte[] key) {
+        synchronized (mux) {
+            return doGet(key, LATEST_REV, false);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Entry get(byte[] key, long rev) {
+        synchronized (mux) {
+            return doGet(key, rev, true);
+        }
+    }
+
+    /** {@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);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(byte[] key) {
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            if (doRemove(key, curRev))
+                rev = curRev;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Entry getAndRemove(byte[] key) {
+        synchronized (mux) {
+            Entry e = doGet(key, LATEST_REV, false);
+
+            if (e.empty() || e.tombstone())
+                return e;
+
+            return getAndPut(key, TOMBSTONE);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(List<byte[]> keys) {
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            List<byte[]> existingKeys = new ArrayList<>(keys.size());
+
+            List<byte[]> vals = new ArrayList<>(keys.size());
+
+            for (byte[] key : keys) {
+                Entry e = doGet(key, LATEST_REV, false);
+
+                if (e.empty() || e.tombstone())
+                    continue;
+
+                existingKeys.add(key);
+
+                vals.add(TOMBSTONE);
+            }
+
+            doPutAll(curRev, existingKeys, vals);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @NotNull
+    @Override public Collection<Entry> getAndRemoveAll(List<byte[]> keys) {
+        Collection<Entry> res = new ArrayList<>(keys.size());
+
+        synchronized (mux) {
+            long curRev = rev + 1;
+
+            List<byte[]> existingKeys = new ArrayList<>(keys.size());
+
+            List<byte[]> vals = new ArrayList<>(keys.size());
+
+            for (byte[] key : keys) {
+                Entry e = doGet(key, LATEST_REV, false);
+
+                res.add(e);
+
+                if (e.empty() || e.tombstone())
+                    continue;
+
+                existingKeys.add(key);
+
+                vals.add(TOMBSTONE);
+            }
+
+            doPutAll(curRev, existingKeys, vals);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure) {
+        synchronized (mux) {
+            Entry e = get(condition.key());
+
+            boolean branch = condition.test(e);
+
+            Collection<Operation> ops = branch ? success : failure;
+
+            long curRev = rev + 1;
+
+            boolean modified = false;
+
+            for (Operation op : ops) {
+                switch (op.type()) {
+                    case PUT:
+                        doPut(op.key(), op.value(), curRev);
+
+                        modified = true;
+
+                        break;
+
+                    case REMOVE:
+                        modified |= doRemove(op.key(), curRev);
+
+                        break;
+
+                    case NO_OP:
+                        break;
+
+                    default:
+                        throw new IllegalArgumentException("Unknown operation type: " + op.type());
+                }
+            }
+
+            if (modified)
+                rev = curRev;
+
+            return branch;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo) {
+        return new RangeCursor(keyFrom, keyTo, rev);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+        return new RangeCursor(keyFrom, keyTo, revUpperBound);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
+        assert keyFrom != null;
+        assert rev > 0;
+
+        return new WatchCursor(rev, k ->
+            CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<WatchEvent> watch(byte[] key, long rev) {
+        assert key != null;
+        assert rev > 0;
+
+        return new WatchCursor(rev, k -> CMP.compare(k, key) == 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+        assert keys != null && !keys.isEmpty();
+        assert rev > 0;
+
+        TreeSet<byte[]> keySet = new TreeSet<>(CMP);
+
+        keySet.addAll(keys);
+
+        return new WatchCursor(rev, keySet::contains);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void compact() {
+        synchronized (mux) {
+            NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(CMP);
+
+            NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new TreeMap<>();
+
+            keysIdx.forEach((key, revs) -> compactForKey(key, revs, compactedKeysIdx, compactedRevsIdx));
+
+            keysIdx = compactedKeysIdx;
+
+            revsIdx = compactedRevsIdx;
+        }
+    }
+
+    /** */
+    private boolean doRemove(byte[] key, long curRev) {
+        Entry e = doGet(key, LATEST_REV, false);
+
+        if (e.empty() || e.tombstone())
+            return false;
+
+        doPut(key, TOMBSTONE, curRev);
+
+        return true;
+    }
+
+    /** */
+    private void compactForKey(
+            byte[] key,
+            List<Long> revs,
+            NavigableMap<byte[], List<Long>> compactedKeysIdx,
+            NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx
+    ) {
+        Long lastRev = lastRevision(revs);
+
+        NavigableMap<byte[], Value> kv = revsIdx.get(lastRev);
+
+        Value lastVal = kv.get(key);
+
+        if (!lastVal.tombstone()) {
+            compactedKeysIdx.put(key, listOf(lastRev));
+
+            NavigableMap<byte[], Value> compactedKv = compactedRevsIdx.computeIfAbsent(
+                    lastRev,
+                    k -> new TreeMap<>(CMP)
+            );
+
+            compactedKv.put(key, lastVal);
+        }
+    }
+
+    /** */
+    @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.";
+        assert rev > 0 || rev == LATEST_REV : "Revision must be positive.";
+
+        Collection<Entry> res = new ArrayList<>(keys.size());
+
+        synchronized (mux) {
+            for (byte[] key : keys) {
+                res.add(doGet(key, rev, false));
+            }
+        }
+
+        return res;
+    }
+
+    /** */
+    @NotNull
+    private Entry doGet(byte[] key, long rev, boolean exactRev) {
+        assert rev == LATEST_REV && !exactRev || rev > LATEST_REV :
+                "Invalid arguments: [rev=" + rev + ", exactRev=" + exactRev + ']';
+
+        List<Long> revs = keysIdx.get(key);
+
+        if (revs == null || revs.isEmpty())
+            return Entry.empty(key);
+
+        long lastRev;
+
+        if (rev == LATEST_REV)
+            lastRev = lastRevision(revs);
+        else
+            lastRev = exactRev ? rev : maxRevision(revs, rev);
+
+        // lastRev can be -1 if maxRevision return -1.
+        if (lastRev == -1)
+            return Entry.empty(key);
+
+        return doGetValue(key, lastRev);
+    }
+
+    /**
+     * 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 upperBoundRev Revision upper bound.
+     * @return Appropriate revision or {@code -1} if there is no such revision.
+     */
+    private static long maxRevision(List<Long> revs, long upperBoundRev) {
+        int i = revs.size() - 1;
+
+        for (; i >= 0; i--) {
+            long rev = revs.get(i);
+
+            if (rev <= upperBoundRev)
+                return rev;
+        }
+
+        return -1;
+    }
+
+    /** */
+    @NotNull
+    private Entry doGetValue(byte[] key, long lastRev) {
+        if (lastRev == 0)
+            return Entry.empty(key);
+
+        NavigableMap<byte[], Value> lastRevVals = revsIdx.get(lastRev);
+
+        if (lastRevVals == null || lastRevVals.isEmpty())
+            return Entry.empty(key);
+
+        Value lastVal = lastRevVals.get(key);
+
+        if (lastVal.tombstone())
+            return Entry.tombstone(key, lastRev, lastVal.updateCounter());
+
+        return new Entry(key, lastVal.bytes(), lastRev, lastVal.updateCounter());
+    }
+
+    /** */
+    private long doPut(byte[] key, byte[] bytes, long curRev) {
+        long curUpdCntr = ++updCntr;
+
+        // Update keysIdx.
+        List<Long> revs = keysIdx.computeIfAbsent(key, k -> new ArrayList<>());
+
+        long lastRev = revs.isEmpty() ? 0 : lastRevision(revs);
+
+        revs.add(curRev);
+
+        // Update revsIdx.
+        Value val = new Value(bytes, curUpdCntr);
+
+        revsIdx.compute(
+                curRev,
+                (rev, entries) -> {
+                    if (entries == null)
+                        entries = new TreeMap<>(CMP);
+
+                    entries.put(key, val);
+
+                    return entries;
+                }
+        );
+
+        return lastRev;
+    }
+
+    /** */
+    private long doPutAll(long curRev, List<byte[]> keys, List<byte[]> bytesList) {
+        synchronized (mux) {
+            // Update revsIdx.
+            NavigableMap<byte[], Value> entries = new TreeMap<>(CMP);
+
+            for (int i = 0; i < keys.size(); i++) {
+                byte[] key = keys.get(i);
+
+                byte[] bytes = bytesList.get(i);
+
+                long curUpdCntr = ++updCntr;
+
+                // Update keysIdx.
+                List<Long> revs = keysIdx.computeIfAbsent(key, k -> new ArrayList<>());
+
+                revs.add(curRev);
+
+                Value val = new Value(bytes, curUpdCntr);
+
+                entries.put(key, val);
+
+                revsIdx.put(curRev, entries);
+            }
+
+            rev = curRev;
+
+            return curRev;
+        }
+    }
+
+    /** */
+    private static long lastRevision(List<Long> revs) {
+        return revs.get(revs.size() - 1);
+    }
+
+    /** */
+    private static List<Long> listOf(long val) {
+        List<Long> res = new ArrayList<>();
+
+        res.add(val);
+
+        return res;
+    }
+
+    /** */
+    private class RangeCursor implements Cursor<Entry> {
+        /** */
+        private final byte[] keyFrom;
+
+        /** */
+        private final byte[] keyTo;
+
+        /** */
+        private final long rev;
+
+        /** */
+        private final Iterator<Entry> it;
+
+        /** */
+        private Entry nextRetEntry;
+
+        /** */
+        private byte[] lastRetKey;
+
+        /** */
+        private boolean finished;
+
+        /** */
+        RangeCursor(byte[] keyFrom, byte[] keyTo, long rev) {
+            this.keyFrom = keyFrom;
+            this.keyTo = keyTo;
+            this.rev = rev;
+            this.it = createIterator();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return it.hasNext();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Entry next() {
+            return it.next();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+            // TODO: implement.
+        }
+
+        /** {@inheritDoc} */
+        @NotNull
+        @Override public Iterator<Entry> iterator() {
+            return it;
+        }
+
+        @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;
+
+                                    break;
+                                }
+
+                                key = e.getKey();
+
+                                if (keyTo != null && CMP.compare(key, keyTo) >= 0) {
+                                    finished = true;
+
+                                    break;
+                                }
+
+                                List<Long> revs = e.getValue();
+
+                                assert revs != null && !revs.isEmpty() :
+                                        "Revisions should not be empty: [revs=" + revs + ']';
+
+                                long lastRev = maxRevision(revs, rev);
+
+                                if (lastRev == -1)
+                                    continue;
+
+                                Entry entry = doGetValue(key, lastRev);
+
+                                assert !entry.empty() : "Iterator should not return empty entry.";
+
+                                nextRetEntry = entry;
+
+                                break;
+                            }
+                        }
+                    }
+                }
+
+                /** {@inheritDoc} */
+                @Override public Entry next() {
+                    synchronized (mux) {
+                        while (true) {
+                            if (finished)
+                                throw new NoSuchElementException();
+
+                            if (nextRetEntry != null) {
+                                Entry e = nextRetEntry;
+
+                                nextRetEntry = null;
+
+                                lastRetKey = e.key();
+
+                                return e;
+                            } else
+                                hasNext();
+                        }
+                    }
+                }
+            };
+        }
+    }
+
+    /** */
+    private class WatchCursor implements Cursor<WatchEvent> {
+        /** */
+        private final Predicate<byte[]> p;
+
+        /** */
+        private final Iterator<WatchEvent> it;
+
+        /** */
+        private long lastRetRev;
+
+        /** */
+        private long nextRetRev = -1;
+
+        /** */
+        WatchCursor(long rev, Predicate<byte[]> p) {
+            this.p = p;
+            this.lastRetRev = rev - 1;
+            this.it = createIterator();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return it.hasNext();
+        }
+
+        /** {@inheritDoc} */
+        @Override public WatchEvent next() {
+            return it.next();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+            // TODO: implement.
+        }
+
+        /** {@inheritDoc} */
+        @NotNull
+        @Override public Iterator<WatchEvent> iterator() {
+            return it;
+        }
+
+        @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 (p.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)
+                                    return null;
+
+                                List<EntryEvent> evts = new ArrayList<>(entries.size());
+
+                                for (Map.Entry<byte[], Value> e : entries.entrySet()) {
+                                    byte[] key = e.getKey();
+
+                                    Value val = e.getValue();
+
+                                    if (p.test(key)) {
+                                        Entry newEntry;
+
+                                        if (val.tombstone())
+                                            newEntry = Entry.tombstone(key, nextRetRev, val.updateCounter());
+                                        else
+                                            newEntry = new Entry(key, val.bytes(), nextRetRev, val.updateCounter());
+
+                                        Entry oldEntry = doGet(key, nextRetRev - 1, false);
+
+                                        evts.add(new EntryEvent(oldEntry, newEntry));
+                                    }
+                                }
+
+                                if (evts.isEmpty())
+                                    continue;
+
+                                lastRetRev = nextRetRev;
+
+                                nextRetRev = -1;
+
+                                return new WatchEvent(evts);
+                            } else if (!hasNext())
+                                return null;
+                        }
+                    }
+                }
+            };
+        }
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
similarity index 50%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java
rename to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
index 7f653cc..ce79b17 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
@@ -15,45 +15,58 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.internal.metastorage.server;
 
 import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Represents a storage unit as entry with key, value and revision, where
- * <ul>
- *     <li>key - an unique entry's key. Keys are comparable in lexicographic manner.</li>
- *     <li>value - a data which is associated with a key and represented as an array of bytes.</li>
- *     <li>revision - a number which denotes a version of whole meta storage. Each change increments the revision.</li>
- * </ul>
+ * Meta storage value with update counter.
  */
-public interface Entry {
+public class Value {
+    /** Tombstone constant. */
+    public static final byte[] TOMBSTONE = new byte[0];
+
+    /** Value. */
+    private final byte[] bytes;
+
+    /** Update counter. */
+    private final long updCntr;
+
     /**
-     * Returns a key.
+     * Constructs value.
      *
-     * @return The key.
+     * @param bytes Value bytes or {@code #TOMBSTONE}.
+     * @param updCntr Update cpunter.
      */
-    @NotNull Key key();
+    public Value(@NotNull byte[] bytes, long updCntr) {
+        this.bytes = bytes;
+        this.updCntr = updCntr;
+    }
 
     /**
-     * Returns a value. Could be {@code null} for empty entry.
+     * Returns value bytes.
      *
-     * @return Value.
+     * @return Value bytes.
      */
-    @Nullable byte[] value();
+    public byte[] bytes() {
+        return bytes;
+    }
 
     /**
-     * Returns a revision.
+     * Returns update counter.
      *
-     * @return Revision.
+     * @return Update counter.
      */
-    long revision();
+    public long updateCounter() {
+        return updCntr;
+    }
 
     /**
-     * Returns an update counter.
+     * Returns {@code true} if value is tombstone.
      *
-     * @return Update counter.
+     * @return {@code True} if value is tombstone, otherwise - {@code false}.
      */
-    long updateCounter();
+    boolean tombstone() {
+        return bytes == TOMBSTONE;
+    }
 }
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ValueCondition.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ValueCondition.java
new file mode 100644
index 0000000..f33ff9b
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/ValueCondition.java
@@ -0,0 +1,82 @@
+/*
+ * 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.Arrays;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Defines condition on entry value.
+ */
+public class ValueCondition extends AbstractCondition {
+    /** Condition type. */
+    @NotNull
+    private final Type type;
+
+    /** Value which will be tested against an entry value. */
+    @NotNull
+    private final byte[] val;
+
+    /**
+     * Constructs value condition with the given type, key and value.
+     *
+     * @param type Condition type. Can't be {@code null}.
+     * @param key Key identifies an entry which condition will be applied to. Can't be {@code null}.
+     * @param val Value which will be tested against an entry value. Can't be {@code null}.
+     */
+    public ValueCondition(@NotNull Type type, @NotNull byte[] key, @NotNull byte[] val) {
+        super(key);
+
+        this.type = type;
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(@NotNull Entry e) {
+        int res = Arrays.compare(e.value(), val);
+
+        return type.test(res);
+    }
+
+    /**
+     * Defines possible condition types which can be applied to the value.
+     */
+    public enum Type {
+        /** Equality condition type. */
+        EQUAL {
+            @Override public boolean test(long res) {
+                return res == 0;
+            }
+        },
+
+        /** Inequality condition type. */
+        NOT_EQUAL {
+            @Override public boolean test(long res) {
+                return res != 0;
+            }
+        };
+
+        /**
+         * Interprets comparison result.
+         *
+         * @param res The result of comparison.
+         * @return The interpretation of the comparison result.
+         */
+        public abstract boolean test(long res);
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java
new file mode 100644
index 0000000..24b1e98
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java
@@ -0,0 +1,72 @@
+/*
+ * 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.Collection;
+import java.util.List;
+
+/**
+ * Watch event contains all entry updates done under one revision. Each particular entry update in this revision
+ * is represented by {@link EntryEvent} entity.
+ */
+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;
+
+    /**
+     * Constructs an watch event with given entry events collection.
+     *
+     * @param entryEvts Events for entries corresponding to an update under one revision.
+     */
+    public WatchEvent(List<EntryEvent> entryEvts) {
+        assert entryEvts != null && !entryEvts.isEmpty();
+
+        this.single = entryEvts.size() == 1;
+        this.entryEvts = entryEvts;
+    }
+
+    /**
+     * Returns {@code true} if watch event contains only one entry event.
+     *
+     * @return {@code True} if watch event contains only one entry event.
+     */
+    public boolean single() {
+        return single;
+    }
+
+    /**
+     * Returns collection of entry entry event done under one revision.
+     *
+     * @return Collection of entry entry event done under one revision.
+     */
+    public Collection<EntryEvent> entryEvents() {
+        return entryEvts;
+    }
+
+    /**
+     * Returns entry event. It is useful method in case when we know that only one event was modified.
+     *
+     * @return Entry event.
+     */
+    public EntryEvent entryEvent() {
+        return entryEvts.get(0);
+    }
+}
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/raft/MetaStorageCommandListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
similarity index 53%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/raft/MetaStorageCommandListener.java
rename to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
index 464d871..77a5a13 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/raft/MetaStorageCommandListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
@@ -15,47 +15,54 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common.raft;
+package org.apache.ignite.internal.metastorage.server.raft;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
+import org.apache.ignite.internal.metastorage.common.ConditionType;
+import org.apache.ignite.internal.metastorage.common.command.ConditionInfo;
 import org.apache.ignite.internal.metastorage.common.command.GetAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndPutAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndPutCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndRemoveAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetAndRemoveCommand;
 import org.apache.ignite.internal.metastorage.common.command.GetCommand;
+import org.apache.ignite.internal.metastorage.common.command.InvokeCommand;
+import org.apache.ignite.internal.metastorage.common.command.MultipleEntryResponse;
+import org.apache.ignite.internal.metastorage.common.command.OperationInfo;
 import org.apache.ignite.internal.metastorage.common.command.PutAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.PutCommand;
 import org.apache.ignite.internal.metastorage.common.command.RangeCommand;
 import org.apache.ignite.internal.metastorage.common.command.RemoveAllCommand;
 import org.apache.ignite.internal.metastorage.common.command.RemoveCommand;
+import org.apache.ignite.internal.metastorage.common.command.SingleEntryResponse;
 import org.apache.ignite.internal.metastorage.common.command.WatchExactKeysCommand;
 import org.apache.ignite.internal.metastorage.common.command.WatchRangeKeysCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorCloseCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorHasNextCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorNextCommand;
+import org.apache.ignite.internal.metastorage.server.Condition;
+import org.apache.ignite.internal.metastorage.server.Entry;
+import org.apache.ignite.internal.metastorage.server.EntryEvent;
+import org.apache.ignite.internal.metastorage.server.ExistenceCondition;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.Operation;
+import org.apache.ignite.internal.metastorage.server.RevisionCondition;
+import org.apache.ignite.internal.metastorage.server.ValueCondition;
+import org.apache.ignite.internal.metastorage.server.WatchEvent;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.metastorage.common.CompactedException;
-import org.apache.ignite.metastorage.common.Cursor;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.KeyValueStorage;
-import org.apache.ignite.metastorage.common.OperationTimeoutException;
-import org.apache.ignite.metastorage.common.WatchEvent;
 import org.apache.ignite.raft.client.ReadCommand;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.apache.ignite.raft.client.service.CommandClosure;
 import org.apache.ignite.raft.client.service.RaftGroupCommandListener;
-import org.jetbrains.annotations.NotNull;
 
 /**
  * Meta storage command listener.
@@ -68,7 +75,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
     private final KeyValueStorage storage;
 
     /** Cursors map. */
-    private final Map<IgniteUuid, IgniteBiTuple<@NotNull Cursor, @NotNull Iterator>> cursors;
+    private final Map<IgniteUuid, IgniteBiTuple<Cursor<?>, CursorType>> cursors;
 
     /**
      * @param storage Storage.
@@ -87,37 +94,47 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 if (clo.command() instanceof GetCommand) {
                     GetCommand getCmd = (GetCommand)clo.command();
 
-                    if (getCmd.revision() != null)
-                        clo.success(storage.get(getCmd.key().bytes(), getCmd.revision()));
+                    Entry e;
+
+                    if (getCmd.revision() != 0)
+                        e = storage.get(getCmd.key(), getCmd.revision());
                     else
-                        clo.success(storage.get(getCmd.key().bytes()));
+                        e = storage.get(getCmd.key());
+
+                    SingleEntryResponse resp = new SingleEntryResponse(
+                            e.key(), e.value(), e.revision(), e.updateCounter()
+                    );
+
+                    clo.success(resp);
                 }
                 else if (clo.command() instanceof GetAllCommand) {
                     GetAllCommand getAllCmd = (GetAllCommand)clo.command();
 
-                    if (getAllCmd.revision() != null) {
-                        clo.success(storage.getAll(
-                            getAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()),
-                            getAllCmd.revision())
-                        );
-                    }
-                    else {
-                        clo.success(storage.getAll(
-                            getAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()))
-                        );
-                    }
+                    Collection<Entry> entries;
+
+                    if (getAllCmd.revision() != 0)
+                        entries = storage.getAll(getAllCmd.keys(), getAllCmd.revision());
+                    else
+                        entries = storage.getAll(getAllCmd.keys());
+
+                    List<SingleEntryResponse> res = new ArrayList<>(entries.size());
+
+                    for (Entry e : entries)
+                        res.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+
+                    clo.success(new MultipleEntryResponse(res));
                 }
                 else if (clo.command() instanceof CursorHasNextCommand) {
                     CursorHasNextCommand cursorHasNextCmd = (CursorHasNextCommand)clo.command();
 
                     assert cursors.containsKey(cursorHasNextCmd.cursorId());
 
-                    clo.success(cursors.get(cursorHasNextCmd.cursorId()).getValue().hasNext());
+                    clo.success(cursors.get(cursorHasNextCmd.cursorId()).getKey().hasNext());
                 }
                 else
                     assert false : "Command was not found [cmd=" + clo.command() + ']';
             }
-            catch (CompactedException | OperationTimeoutException e) {
+            catch (Exception e) {
                 // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
                 LOG.warn("Unable to evaluate command [cmd=" + clo.command() + ']', e);
 
@@ -140,67 +157,79 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 if (clo.command() instanceof PutCommand) {
                     PutCommand putCmd = (PutCommand)clo.command();
 
-                    storage.put(putCmd.key().bytes(), putCmd.value());
+                    storage.put(putCmd.key(), putCmd.value());
 
                     clo.success(null);
                 }
                 else if (clo.command() instanceof GetAndPutCommand) {
                     GetAndPutCommand getAndPutCmd = (GetAndPutCommand)clo.command();
 
-                    clo.success(storage.getAndPut(getAndPutCmd.key().bytes(), getAndPutCmd.value()));
+                    Entry e = storage.getAndPut(getAndPutCmd.key(), getAndPutCmd.value());
+
+                    clo.success(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
                 }
                 else if (clo.command() instanceof PutAllCommand) {
                     PutAllCommand putAllCmd = (PutAllCommand)clo.command();
 
-                    storage.putAll(
-                        putAllCmd.values().keySet().stream().map(Key::bytes).collect(Collectors.toList()),
-                        new ArrayList<>(putAllCmd.values().values()));
+                    storage.putAll(putAllCmd.keys(), putAllCmd.values());
 
                     clo.success(null);
                 }
                 else if (clo.command() instanceof GetAndPutAllCommand) {
                     GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand)clo.command();
 
-                    Collection<Entry> entries = storage.getAndPutAll(
-                        getAndPutAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()),
-                        getAndPutAllCmd.vals()
-                    );
+                    Collection<Entry> entries = storage.getAndPutAll(getAndPutAllCmd.keys(), getAndPutAllCmd.vals());
+
+                    List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
 
-                    if (!(entries instanceof Serializable))
-                        entries = new ArrayList<>(entries);
+                    for (Entry e : entries)
+                        resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
 
-                    clo.success(entries);
+                    clo.success(new MultipleEntryResponse(resp));
                 }
                 else if (clo.command() instanceof RemoveCommand) {
                     RemoveCommand rmvCmd = (RemoveCommand)clo.command();
 
-                    storage.remove(rmvCmd.key().bytes());
+                    storage.remove(rmvCmd.key());
 
                     clo.success(null);
                 }
                 else if (clo.command() instanceof GetAndRemoveCommand) {
                     GetAndRemoveCommand getAndRmvCmd = (GetAndRemoveCommand)clo.command();
 
-                    clo.success(storage.getAndRemove(getAndRmvCmd.key().bytes()));
+                    Entry e = storage.getAndRemove(getAndRmvCmd.key());
+
+                    clo.success(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
                 }
                 else if (clo.command() instanceof RemoveAllCommand) {
                     RemoveAllCommand rmvAllCmd = (RemoveAllCommand)clo.command();
 
-                    storage.removeAll(rmvAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()));
+                    storage.removeAll(rmvAllCmd.keys());
 
                     clo.success(null);
                 }
                 else if (clo.command() instanceof GetAndRemoveAllCommand) {
                     GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand)clo.command();
 
-                    Collection<Entry> entries = storage.getAndRemoveAll(
-                        getAndRmvAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList())
-                    );
+                    Collection<Entry> entries = storage.getAndRemoveAll(getAndRmvAllCmd.keys());
+
+                    List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
 
-                    if (!(entries instanceof Serializable))
-                        entries = new ArrayList<>(entries);
+                    for (Entry e : entries)
+                        resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
 
-                    clo.success(entries);
+                    clo.success(new MultipleEntryResponse(resp));
+                }
+                else if (clo.command() instanceof InvokeCommand) {
+                    InvokeCommand cmd = (InvokeCommand)clo.command();
+
+                    boolean res = storage.invoke(
+                            toCondition(cmd.condition()),
+                            toOperations(cmd.success()),
+                            toOperations(cmd.failure())
+                    );
+
+                    clo.success(res);
                 }
                 else if (clo.command() instanceof RangeCommand) {
                     RangeCommand rangeCmd = (RangeCommand)clo.command();
@@ -208,15 +237,12 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                     IgniteUuid cursorId = new IgniteUuid(UUID.randomUUID(), 0L);
 
                     Cursor<Entry> cursor = storage.range(
-                        rangeCmd.keyFrom().bytes(),
-                        rangeCmd.keyTo() == null ? null : rangeCmd.keyTo().bytes(),
+                        rangeCmd.keyFrom(),
+                        rangeCmd.keyTo(),
                         rangeCmd.revUpperBound()
                     );
 
-                    cursors.put(
-                        cursorId,
-                        new IgniteBiTuple<>(cursor, cursor.iterator())
-                    );
+                    cursors.put(cursorId, new IgniteBiTuple<>(cursor, CursorType.RANGE));
 
                     clo.success(cursorId);
                 }
@@ -225,7 +251,31 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
 
                     assert cursors.containsKey(cursorNextCmd.cursorId());
 
-                    clo.success(cursors.get(cursorNextCmd.cursorId()).getValue().next());
+                    IgniteBiTuple<Cursor<?>, CursorType> cursorDesc = cursors.get(cursorNextCmd.cursorId());
+
+                    if (cursorDesc.getValue() == CursorType.RANGE) {
+                        Entry e = (Entry)cursorDesc.getKey().next();
+
+                        clo.success(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+                    }
+                    else if (cursorDesc.getValue() == CursorType.WATCH) {
+                        WatchEvent evt = (WatchEvent)cursorDesc.getKey().next();
+
+                        List<SingleEntryResponse> resp = new ArrayList<>(evt.entryEvents().size() * 2);
+
+                        for (EntryEvent e : evt.entryEvents()) {
+                            Entry o = e.oldEntry();
+
+                            Entry n = e.entry();
+
+                            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.success(new MultipleEntryResponse(resp));
+                    }
+
                 }
                 else if (clo.command() instanceof CursorCloseCommand) {
                     CursorCloseCommand cursorCloseCmd = (CursorCloseCommand)clo.command();
@@ -250,15 +300,10 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
 
                     IgniteUuid cursorId = new IgniteUuid(UUID.randomUUID(), 0L);
 
-                    Cursor<WatchEvent> cursor = storage.watch(
-                        watchCmd.keyFrom() == null ? null : watchCmd.keyFrom().bytes(),
-                        watchCmd.keyTo() == null ? null : watchCmd.keyTo().bytes(),
-                        watchCmd.revision());
+                    Cursor<WatchEvent> cursor =
+                            storage.watch(watchCmd.keyFrom(), watchCmd.keyTo(), watchCmd.revision());
 
-                    cursors.put(
-                        cursorId,
-                        new IgniteBiTuple<>(cursor, cursor.iterator())
-                    );
+                    cursors.put(cursorId, new IgniteBiTuple<>(cursor, CursorType.WATCH));
 
                     clo.success(cursorId);
                 }
@@ -267,21 +312,16 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
 
                     IgniteUuid cursorId = new IgniteUuid(UUID.randomUUID(), 0L);
 
-                    Cursor<WatchEvent> cursor = storage.watch(
-                        watchCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()),
-                        watchCmd.revision());
+                    Cursor<WatchEvent> cursor = storage.watch(watchCmd.keys(), watchCmd.revision());
 
-                    cursors.put(
-                        cursorId,
-                        new IgniteBiTuple<>(cursor, cursor.iterator())
-                    );
+                    cursors.put(cursorId, new IgniteBiTuple<>(cursor, CursorType.WATCH));
 
                     clo.success(cursorId);
                 }
                 else
                     assert false : "Command was not found [cmd=" + clo.command() + ']';
             }
-            catch (CompactedException | OperationTimeoutException e) {
+            catch (Exception e) {
                 // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
                 LOG.warn("Unable to evaluate command [cmd=" + clo.command() + ']', e);
 
@@ -294,4 +334,49 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
             }
         }
     }
+
+    /** */
+    private static Condition toCondition(ConditionInfo info) {
+        byte[] key = info.key();
+
+        ConditionType type = info.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.VAL_EQUAL)
+            return new ValueCondition(ValueCondition.Type.EQUAL, key, info.value());
+        else if (type == ConditionType.VAL_NOT_EQUAL)
+            return new ValueCondition(ValueCondition.Type.NOT_EQUAL, key, info.value());
+        else if (type == ConditionType.REV_EQUAL)
+            return new RevisionCondition(RevisionCondition.Type.EQUAL, key, info.revision());
+        else if (type == ConditionType.REV_NOT_EQUAL)
+            return new RevisionCondition(RevisionCondition.Type.NOT_EQUAL, key, info.revision());
+        else if (type == ConditionType.REV_GREATER)
+            return new RevisionCondition(RevisionCondition.Type.GREATER, key, info.revision());
+        else if (type == ConditionType.REV_GREATER_OR_EQUAL)
+            return new RevisionCondition(RevisionCondition.Type.GREATER_OR_EQUAL, key, info.revision());
+        else if (type == ConditionType.REV_LESS)
+            return new RevisionCondition(RevisionCondition.Type.LESS, key, info.revision());
+        else if (type == ConditionType.REV_LESS_OR_EQUAL)
+            return new RevisionCondition(RevisionCondition.Type.LESS_OR_EQUAL, key, info.revision());
+        else
+            throw new IllegalArgumentException("Unknown condition type: " + type);
+    }
+
+    /** */
+    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 type. */
+    private enum CursorType {
+        RANGE, WATCH;
+    }
 }
diff --git a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ExistenceConditionTest.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ExistenceConditionTest.java
new file mode 100644
index 0000000..ec4a573
--- /dev/null
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ExistenceConditionTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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 org.junit.jupiter.api.Test;
+
+import static org.apache.ignite.internal.metastorage.server.ExistenceCondition.Type.EXISTS;
+import static org.apache.ignite.internal.metastorage.server.ExistenceCondition.Type.NOT_EXISTS;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for entry existence condition.
+ *
+ * @see ExistenceCondition
+ */
+public class ExistenceConditionTest {
+    /** Entry key. */
+    private static final byte[] KEY = new byte[] {1};
+
+    /** Entry value. */
+    private static final byte[] VAL = new byte[] {1};
+
+    /** Regular entry. */
+    private static final Entry ENTRY = new Entry(KEY, VAL, 1, 1);
+
+    /** Empty entry. */
+    private static final Entry EMPTY = Entry.empty(KEY);
+
+    /** Tombstone entry. */
+    private static final Entry TOMBSTONE = Entry.tombstone(KEY, 1, 1);
+
+    /**
+     * Tests {@link ExistenceCondition.Type#EXISTS} condition for regular, empty and tombstone entries.
+     */
+    @Test
+    public void exists() {
+        Condition cond = new ExistenceCondition(EXISTS, KEY);
+
+        assertTrue(cond.test(ENTRY));
+        assertFalse(cond.test(EMPTY));
+        assertFalse(cond.test(TOMBSTONE));
+    }
+
+    /**
+     * Tests {@link ExistenceCondition.Type#NOT_EXISTS} condition for regular, empty and tombstone entries.
+     */
+    @Test
+    public void notExists() {
+        Condition cond = new ExistenceCondition(NOT_EXISTS, KEY);
+
+        assertFalse(cond.test(ENTRY));
+        assertTrue(cond.test(EMPTY));
+        assertTrue(cond.test(TOMBSTONE));
+    }
+}
diff --git a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/RevisionConditionTest.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/RevisionConditionTest.java
new file mode 100644
index 0000000..2c461aa
--- /dev/null
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/RevisionConditionTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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 org.junit.jupiter.api.Test;
+
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.EQUAL;
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.GREATER;
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.LESS;
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.LESS_OR_EQUAL;
+import static org.apache.ignite.internal.metastorage.server.RevisionCondition.Type.NOT_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for entry revision condition.
+ *
+ * @see RevisionCondition
+ */
+public class RevisionConditionTest {
+    /** Entry key. */
+    private static final byte[] KEY = new byte[] {1};
+
+    /** Entry value. */
+    private static final byte[] VAL = new byte[] {2};
+
+    /**
+     * Tests revisions equality.
+     */
+    @Test
+    public void eq() {
+        Condition cond = new RevisionCondition(EQUAL, KEY, 1);
+
+        // 1 == 1.
+        assertTrue(cond.test(new Entry(KEY, VAL, 1, 1)));
+    }
+
+    /**
+     * Tests revisions inequality.
+     */
+    @Test
+    public void ne() {
+        Condition cond = new RevisionCondition(NOT_EQUAL, KEY, 1);
+
+        // 2 != 1.
+        assertTrue(cond.test(new Entry(KEY, VAL, 2, 1)));
+    }
+
+    /**
+     * Tests that revision is greater than another one.
+     */
+    @Test
+    public void gt() {
+        Condition cond = new RevisionCondition(GREATER, KEY, 1);
+
+        // 2 > 1.
+        assertTrue(cond.test(new Entry(KEY, VAL, 2, 1)));
+    }
+
+    /**
+     * Tests that revision is greater than or equal to another one .
+     */
+    @Test
+    public void ge() {
+        Condition cond = new RevisionCondition(GREATER_OR_EQUAL, KEY, 1);
+
+        // 2 >= 1 (2 > 1).
+        assertTrue(cond.test(new Entry(KEY, VAL, 2, 1)));
+
+        // 1 >= 1 (1 == 1).
+        assertTrue(cond.test(new Entry(KEY, VAL, 1, 1)));
+    }
+
+    /**
+     * Tests that revision is less than another one.
+     */
+    @Test
+    public void lt() {
+        Condition cond = new RevisionCondition(LESS, KEY, 2);
+
+        // 1 < 2
+        assertTrue(cond.test(new Entry(KEY, VAL, 1, 1)));
+    }
+
+    /**
+     * Tests that revision is less than or equal to another one .
+     */
+    @Test
+    public void le() {
+        Condition cond = new RevisionCondition(LESS_OR_EQUAL, KEY, 2);
+
+        // 1 <= 2 (1 < 2)
+        assertTrue(cond.test(new Entry(KEY, VAL, 1, 1)));
+
+        // 1 <= 1 (1 == 1).
+        assertTrue(cond.test(new Entry(KEY, VAL, 1, 1)));
+    }
+}
diff --git a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
new file mode 100644
index 0000000..342b4ba
--- /dev/null
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
@@ -0,0 +1,1945 @@
+/*
+ * 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.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.ByteArray;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import static java.util.function.Function.identity;
+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.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * Tests for in-memory meta storage implementation.
+ */
+class SimpleInMemoryKeyValueStorageTest {
+    /** */
+    private KeyValueStorage storage;
+
+    @BeforeEach
+    public void setUp() {
+        storage = new SimpleInMemoryKeyValueStorage();
+    }
+
+    @Test
+    public void put() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        storage.put(key, val);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        Entry e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertFalse(e.tombstone());
+        assertEquals(1, e.revision());
+        assertEquals(1, e.updateCounter());
+
+        storage.put(key, val);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertFalse(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+    }
+
+    @Test
+    void getAll() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Regular put.
+        storage.put(key1, val1);
+
+        // Rewrite.
+        storage.put(key2, val2_1);
+        storage.put(key2, val2_2);
+
+        // Remove.
+        storage.put(key3, val3);
+        storage.remove(key3);
+
+        assertEquals(5, storage.revision());
+        assertEquals(5, storage.updateCounter());
+
+        Collection<Entry> entries = storage.getAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(4, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+        assertArrayEquals(val1, e1.value());
+
+        // Test rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(3, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_2, e2.value());
+
+        // Test removed value.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(5, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    void getAllWithRevisionBound() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Regular put.
+        storage.put(key1, val1);
+
+        // Rewrite.
+        storage.put(key2, val2_1);
+        storage.put(key2, val2_2);
+
+        // Remove.
+        storage.put(key3, val3);
+        storage.remove(key3);
+
+        assertEquals(5, storage.revision());
+        assertEquals(5, storage.updateCounter());
+
+        // Bounded by revision 2.
+        Collection<Entry> entries = storage.getAll(List.of(key1, key2, key3, key4), 2);
+
+        assertEquals(4, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+        assertArrayEquals(val1, e1.value());
+
+        // Test while not rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(2, e2.revision());
+        assertEquals(2, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_1, e2.value());
+
+        // Values with larger revision don't exist yet.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertTrue(e3.empty());
+
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertTrue(e4.empty());
+
+        // Bounded by revision 4.
+        entries = storage.getAll(List.of(key1, key2, key3, key4), 4);
+
+        assertEquals(4, entries.size());
+
+        map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+        assertArrayEquals(val1, e1.value());
+
+        // Test rewritten value.
+        e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(3, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_2, e2.value());
+
+        // Test not removed value.
+        e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(4, e3.revision());
+        assertEquals(4, e3.updateCounter());
+        assertFalse(e3.tombstone());
+        assertFalse(e3.empty());
+        assertArrayEquals(val3, e3.value());
+
+        // Value with larger revision doesn't exist yet.
+        e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    public void getAndPut() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        Entry e = storage.getAndPut(key, val);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+        assertTrue(e.empty());
+        assertFalse(e.tombstone());
+        assertEquals(0, e.revision());
+        assertEquals(0, e.updateCounter());
+
+        e = storage.getAndPut(key, val);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+        assertFalse(e.empty());
+        assertFalse(e.tombstone());
+        assertEquals(1, e.revision());
+        assertEquals(1, e.updateCounter());
+    }
+
+    @Test
+    public void putAll() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3_1 = kv(3, 31);
+        byte[] val3_2 = kv(3, 32);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Must be rewritten.
+        storage.put(key2, val2_1);
+
+        // Remove. Tombstone must be replaced by new value.
+        storage.put(key3, val3_1);
+        storage.remove(key3);
+
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        storage.putAll(List.of(key1, key2, key3), List.of(val1, val2_2, val3_2));
+
+        assertEquals(4, storage.revision());
+        assertEquals(6, storage.updateCounter());
+
+        Collection<Entry> entries = storage.getAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(4, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(4, e1.revision());
+        assertEquals(4, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+        assertArrayEquals(val1, e1.value());
+
+        // Test rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(4, e2.revision());
+        assertEquals(5, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_2, e2.value());
+
+        // Test removed value.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(4, e3.revision());
+        assertEquals(6, e3.updateCounter());
+        assertFalse(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    public void getAndPutAll() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3_1 = kv(3, 31);
+        byte[] val3_2 = kv(3, 32);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Must be rewritten.
+        storage.put(key2, val2_1);
+
+        // Remove. Tombstone must be replaced by new value.
+        storage.put(key3, val3_1);
+        storage.remove(key3);
+
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Collection<Entry> entries = storage.getAndPutAll(List.of(key1, key2, key3), List.of(val1, val2_2, val3_2));
+
+        assertEquals(4, storage.revision());
+        assertEquals(6, storage.updateCounter());
+
+        assertEquals(3, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(0, e1.revision());
+        assertEquals(0, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertTrue(e1.empty());
+
+        // Test rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(1, e2.revision());
+        assertEquals(1, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_1, e2.value());
+
+        // Test removed value.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(3, e3.revision());
+        assertEquals(3, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test state after putAll.
+        entries = storage.getAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(4, entries.size());
+
+        map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(4, e1.revision());
+        assertEquals(4, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+        assertArrayEquals(val1, e1.value());
+
+        // Test rewritten value.
+        e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(4, e2.revision());
+        assertEquals(5, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+        assertArrayEquals(val2_2, e2.value());
+
+        // Test removed value.
+        e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(4, e3.revision());
+        assertEquals(6, e3.updateCounter());
+        assertFalse(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    public void remove() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        // Remove non-existent entry.
+        storage.remove(key);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        storage.put(key, val);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        // Remove existent entry.
+        storage.remove(key);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        Entry e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertTrue(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+
+        // Remove already removed entry (tombstone can't be removed).
+        storage.remove(key);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertTrue(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+    }
+
+    @Test
+    public void getAndRemove() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        // Remove non-existent entry.
+        Entry e = storage.getAndRemove(key);
+
+        assertTrue(e.empty());
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+        assertTrue(storage.get(key).empty());
+
+        storage.put(key, val);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        // Remove existent entry.
+        e = storage.getAndRemove(key);
+
+        assertFalse(e.empty());
+        assertFalse(e.tombstone());
+        assertEquals(1, e.revision());
+        assertEquals(1, e.updateCounter());
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertTrue(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+
+        // Remove already removed entry (tombstone can't be removed).
+        e = storage.getAndRemove(key);
+
+        assertFalse(e.empty());
+        assertTrue(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        e = storage.get(key);
+
+        assertFalse(e.empty());
+        assertTrue(e.tombstone());
+        assertEquals(2, e.revision());
+        assertEquals(2, e.updateCounter());
+    }
+
+    @Test
+    public void removeAll() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3_1 = kv(3, 31);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Regular put.
+        storage.put(key1, val1);
+
+        // Rewrite.
+        storage.put(key2, val2_1);
+        storage.put(key2, val2_2);
+
+        // Remove. Tombstone must not be removed again.
+        storage.put(key3, val3_1);
+        storage.remove(key3);
+
+        assertEquals(5, storage.revision());
+        assertEquals(5, storage.updateCounter());
+
+        storage.removeAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(6, storage.revision());
+        assertEquals(7, storage.updateCounter()); // Only two keys are updated.
+
+        Collection<Entry> entries = storage.getAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(4, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(6, e1.revision());
+        assertEquals(6, e1.updateCounter());
+        assertTrue(e1.tombstone());
+        assertFalse(e1.empty());
+
+        // Test rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(6, e2.revision());
+        assertEquals(7, e2.updateCounter());
+        assertTrue(e2.tombstone());
+        assertFalse(e2.empty());
+
+        // Test removed value.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(5, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    public void getAndRemoveAll() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+        byte[] key3 = k(3);
+        byte[] val3_1 = kv(3, 31);
+
+        byte[] key4 = k(4);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Regular put.
+        storage.put(key1, val1);
+
+        // Rewrite.
+        storage.put(key2, val2_1);
+        storage.put(key2, val2_2);
+
+        // Remove. Tombstone must not be removed again.
+        storage.put(key3, val3_1);
+        storage.remove(key3);
+
+        assertEquals(5, storage.revision());
+        assertEquals(5, storage.updateCounter());
+
+        Collection<Entry> entries = storage.getAndRemoveAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(6, storage.revision());
+        assertEquals(7, storage.updateCounter()); // Only two keys are updated.
+
+        assertEquals(4, entries.size());
+
+        Map<ByteArray, Entry> map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+        assertFalse(e1.tombstone());
+        assertFalse(e1.empty());
+
+        // Test rewritten value.
+        Entry e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(3, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertFalse(e2.tombstone());
+        assertFalse(e2.empty());
+
+        // Test removed value.
+        Entry e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(5, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        Entry e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+
+        // Test state after getAndRemoveAll.
+        entries = storage.getAll(List.of(key1, key2, key3, key4));
+
+        assertEquals(4, entries.size());
+
+        map = entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
+
+        // Test regular put value.
+        e1 = map.get(new ByteArray(key1));
+
+        assertNotNull(e1);
+        assertEquals(6, e1.revision());
+        assertEquals(6, e1.updateCounter());
+        assertTrue(e1.tombstone());
+        assertFalse(e1.empty());
+
+        // Test rewritten value.
+        e2 = map.get(new ByteArray(key2));
+
+        assertNotNull(e2);
+        assertEquals(6, e2.revision());
+        assertEquals(7, e2.updateCounter());
+        assertTrue(e2.tombstone());
+        assertFalse(e2.empty());
+
+        // Test removed value.
+        e3 = map.get(new ByteArray(key3));
+
+        assertNotNull(e3);
+        assertEquals(5, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        e4 = map.get(new ByteArray(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
+    public void getAfterRemove() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        storage.getAndPut(key, val);
+
+        storage.getAndRemove(key);
+
+        Entry e = storage.get(key);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+        assertEquals(2, e.revision());
+        assertTrue(e.tombstone());
+    }
+
+    @Test
+    public void getAndPutAfterRemove() {
+        byte[] key = k(1);
+        byte[] val = kv(1, 1);
+
+        storage.getAndPut(key, val);
+
+        storage.getAndRemove(key);
+
+        Entry e = storage.getAndPut(key, val);
+
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+        assertEquals(2, e.revision());
+        assertTrue(e.tombstone());
+    }
+
+    @Test
+    public void putGetRemoveCompact() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 1);
+        byte[] val1_3 = kv(1, 3);
+
+        byte[] key2 = k(2);
+        byte[] val2_2 = kv(2, 2);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Previous entry is empty.
+        Entry emptyEntry = storage.getAndPut(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+        assertTrue(emptyEntry.empty());
+
+        // Entry with rev == 1.
+        Entry e1_1 = storage.get(key1);
+
+        assertFalse(e1_1.empty());
+        assertFalse(e1_1.tombstone());
+        assertArrayEquals(key1, e1_1.key());
+        assertArrayEquals(val1_1, e1_1.value());
+        assertEquals(1, e1_1.revision());
+        assertEquals(1, e1_1.updateCounter());
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        // Previous entry is empty.
+        emptyEntry = storage.getAndPut(key2, val2_2);
+
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+        assertTrue(emptyEntry.empty());
+
+        // Entry with rev == 2.
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertArrayEquals(key2, e2.key());
+        assertArrayEquals(val2_2, e2.value());
+        assertEquals(2, e2.revision());
+        assertEquals(2, e2.updateCounter());
+        assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        // Previous entry is not empty.
+        e1_1 = storage.getAndPut(key1, val1_3);
+
+        assertFalse(e1_1.empty());
+        assertFalse(e1_1.tombstone());
+        assertArrayEquals(key1, e1_1.key());
+        assertArrayEquals(val1_1, e1_1.value());
+        assertEquals(1, e1_1.revision());
+        assertEquals(1, e1_1.updateCounter());
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        // Entry with rev == 3.
+        Entry e1_3 = storage.get(key1);
+
+        assertFalse(e1_3.empty());
+        assertFalse(e1_3.tombstone());
+        assertArrayEquals(key1, e1_3.key());
+        assertArrayEquals(val1_3, e1_3.value());
+        assertEquals(3, e1_3.revision());
+        assertEquals(3, e1_3.updateCounter());
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        // Remove existing entry.
+        Entry e2_2 = storage.getAndRemove(key2);
+
+        assertFalse(e2_2.empty());
+        assertFalse(e2_2.tombstone());
+        assertArrayEquals(key2, e2_2.key());
+        assertArrayEquals(val2_2, e2_2.value());
+        assertEquals(2, e2_2.revision());
+        assertEquals(2, e2_2.updateCounter());
+        assertEquals(4, storage.revision()); // Storage revision is changed.
+        assertEquals(4, storage.updateCounter());
+
+        // Remove already removed entry.
+        Entry tombstoneEntry = storage.getAndRemove(key2);
+
+        assertFalse(tombstoneEntry.empty());
+        assertTrue(tombstoneEntry.tombstone());
+        assertEquals(4, storage.revision()); // Storage revision is not changed.
+        assertEquals(4, storage.updateCounter());
+
+        // Compact and check that tombstones are removed.
+        storage.compact();
+
+        assertEquals(4, storage.revision());
+        assertEquals(4, storage.updateCounter());
+        assertTrue(storage.getAndRemove(key2).empty());
+        assertTrue(storage.get(key2).empty());
+
+        // Remove existing entry.
+        e1_3 = storage.getAndRemove(key1);
+
+        assertFalse(e1_3.empty());
+        assertFalse(e1_3.tombstone());
+        assertArrayEquals(key1, e1_3.key());
+        assertArrayEquals(val1_3, e1_3.value());
+        assertEquals(3, e1_3.revision());
+        assertEquals(3, e1_3.updateCounter());
+        assertEquals(5, storage.revision()); // Storage revision is changed.
+        assertEquals(5, storage.updateCounter());
+
+        // Remove already removed entry.
+        tombstoneEntry = storage.getAndRemove(key1);
+
+        assertFalse(tombstoneEntry.empty());
+        assertTrue(tombstoneEntry.tombstone());
+        assertEquals(5, storage.revision()); // // Storage revision is not changed.
+        assertEquals(5, storage.updateCounter());
+
+        // Compact and check that tombstones are removed.
+        storage.compact();
+
+        assertEquals(5, storage.revision());
+        assertEquals(5, storage.updateCounter());
+        assertTrue(storage.getAndRemove(key1).empty());
+        assertTrue(storage.get(key1).empty());
+    }
+
+    @Test
+    public void invokeWithRevisionCondition_successBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new RevisionCondition(RevisionCondition.Type.EQUAL, key1, 1),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                ),
+                List.of(new Operation(OperationType.PUT, key3, val3))
+        );
+
+        // "Success" branch is applied.
+        assertTrue(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Failure" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithRevisionCondition_failureBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new RevisionCondition(RevisionCondition.Type.EQUAL, key1, 2),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                )
+        );
+
+        // "Failure" branch is applied.
+        assertFalse(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Success" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithExistsCondition_successBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ExistenceCondition(ExistenceCondition.Type.EXISTS, key1),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                ),
+                List.of(new Operation(OperationType.PUT, key3, val3))
+        );
+
+        // "Success" branch is applied.
+        assertTrue(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Failure" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithExistsCondition_failureBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ExistenceCondition(ExistenceCondition.Type.EXISTS, key3),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                )
+        );
+
+        // "Failure" branch is applied.
+        assertFalse(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Success" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithNotExistsCondition_successBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key2),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                ),
+                List.of(new Operation(OperationType.PUT, key3, val3))
+        );
+
+        // "Success" branch is applied.
+        assertTrue(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Failure" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithNotExistsCondition_failureBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key1),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                )
+        );
+
+        // "Failure" branch is applied.
+        assertFalse(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Success" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithValueCondition_successBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ValueCondition(ValueCondition.Type.EQUAL, key1, val1_1),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                ),
+                List.of(new Operation(OperationType.PUT, key3, val3))
+        );
+
+        // "Success" branch is applied.
+        assertTrue(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Failure" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeWithValueCondition_failureBranch() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        boolean branch = storage.invoke(
+                new ValueCondition(ValueCondition.Type.EQUAL, key1, val1_2),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
+                List.of(
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
+                )
+        );
+
+        // "Failure" branch is applied.
+        assertFalse(branch);
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e1 = storage.get(key1);
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertEquals(2, e1.revision());
+        assertEquals(2, e1.updateCounter());
+        assertArrayEquals(val1_2, e1.value());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(3, e2.updateCounter());
+        assertArrayEquals(val2, e2.value());
+
+        // "Success" branch isn't applied.
+        Entry e3 = storage.get(key3);
+
+        assertTrue(e3.empty());
+    }
+
+    @Test
+    public void invokeOperations() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
+
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
+
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.put(key1, val1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        // No-op.
+        boolean branch = storage.invoke(
+                new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
+                List.of(new Operation(OperationType.NO_OP, null, null)),
+                List.of(new Operation(OperationType.NO_OP, null, null))
+        );
+
+        assertTrue(branch);
+
+        // No updates.
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        // Put.
+        branch = storage.invoke(
+                new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
+                List.of(
+                        new Operation(OperationType.PUT, key2, val2),
+                        new Operation(OperationType.PUT, key3, val3)
+                ),
+                List.of(new Operation(OperationType.NO_OP, null, null))
+        );
+
+        assertTrue(branch);
+
+        // +1 for revision, +2 for update counter.
+        assertEquals(2, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        Entry e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertEquals(2, e2.revision());
+        assertEquals(2, e2.updateCounter());
+        assertArrayEquals(key2, e2.key());
+        assertArrayEquals(val2, e2.value());
+
+        Entry e3 = storage.get(key3);
+
+        assertFalse(e3.empty());
+        assertFalse(e3.tombstone());
+        assertEquals(2, e3.revision());
+        assertEquals(3, e3.updateCounter());
+        assertArrayEquals(key3, e3.key());
+        assertArrayEquals(val3, e3.value());
+
+        // Remove.
+        branch = storage.invoke(
+                new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
+                List.of(
+                        new Operation(OperationType.REMOVE, key2, null),
+                        new Operation(OperationType.REMOVE, key3, null)
+                ),
+                List.of(new Operation(OperationType.NO_OP, null, null))
+        );
+
+        assertTrue(branch);
+
+        // +1 for revision, +2 for update counter.
+        assertEquals(3, storage.revision());
+        assertEquals(5, storage.updateCounter());
+
+        e2 = storage.get(key2);
+
+        assertFalse(e2.empty());
+        assertTrue(e2.tombstone());
+        assertEquals(3, e2.revision());
+        assertEquals(4, e2.updateCounter());
+        assertArrayEquals(key2, e2.key());
+
+        e3 = storage.get(key3);
+
+        assertFalse(e3.empty());
+        assertTrue(e3.tombstone());
+        assertEquals(3, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertArrayEquals(key3, e3.key());
+    }
+
+    @Test
+    public void compact() {
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Compact empty.
+        storage.compact();
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        // Compact non-empty.
+        fill(storage, 1, 1);
+
+        assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
+
+        fill(storage, 2, 2);
+
+        assertEquals(3, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        fill(storage, 3, 3);
+
+        assertEquals(6, storage.revision());
+        assertEquals(6, storage.updateCounter());
+
+        storage.getAndRemove(k(3));
+
+        assertEquals(7, storage.revision());
+        assertEquals(7, storage.updateCounter());
+        assertTrue(storage.get(k(3)).tombstone());
+
+        storage.compact();
+
+        assertEquals(7, storage.revision());
+        assertEquals(7, storage.updateCounter());
+
+        Entry e1 = storage.get(k(1));
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertArrayEquals(k(1), e1.key());
+        assertArrayEquals(kv(1,1), e1.value());
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+
+        Entry e2 = storage.get(k(2));
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertArrayEquals(k(2), e2.key());
+        assertArrayEquals(kv(2,2), e2.value());
+        assertTrue(storage.get(k(2), 2).empty());
... 1693 lines suppressed ...