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/18 10:36:44 UTC

[ignite-3] branch ignite-14389 updated (6d98268 -> 51d3fb4)

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

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


 discard 6d98268  IGNITE-14389 MetaStorageService integration
 discard 7eebebf  WIP
 discard 5fe099d  IGNITE-14389 Implemented cursor for ranges and watches.
 discard 36722c6  IGNITE-14389 Added putAll and removeAll. Started cursor management: ranges and watches (WIP)
 discard 904f266  IGNITE-14389 putAll initial (WIP)
 discard 02f3371  IGNITE-14389 getAll and tests (WIP)
 discard 9cd33ea  IGNITE-14389 Added get and do smth semantic
 discard ead9ad5  IGNITE-14398: Meta storage: added update counter
 discard 4fbbaa9  IGNITE-14389 Meta storage: in-memory implementation WIP
     add 21ebca1  IGNITE-14707 Fixed topology events processing when nodes are restarted in quick succession (#126).
     add 865a066  IGNITE-14237 Implemented affinity calculation based on events. Fixes #120
     add d2dcab5  IGNITE-14088 ScaleCube transport API over Netty (#125)
     add 9f3e028  IGNITE-14717 Wide use of TestScaleCubeClusterServiceFactory. (#128)
     new a1eb463  IGNITE-14389 Meta storage: in-memory implementation WIP
     new de979bd  IGNITE-14398: Meta storage: added update counter
     new 1a1e8be  IGNITE-14389 Added get and do smth semantic
     new 7e0547d  IGNITE-14389 getAll and tests (WIP)
     new 54b1c42  IGNITE-14389 putAll initial (WIP)
     new efe4f16  IGNITE-14389 Added putAll and removeAll. Started cursor management: ranges and watches (WIP)
     new a0febd6  IGNITE-14389 Implemented cursor for ranges and watches.
     new 67128be  WIP
     new 51d3fb4  IGNITE-14389 MetaStorageService integration

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (6d98268)
            \
             N -- N -- N   refs/heads/ignite-14389 (51d3fb4)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../ignite/internal/affinity/AffinityManager.java  | 168 ++---
 .../affinity/RendezvousAffinityFunction.java       |  11 +-
 .../internal/affinity/event/AffinityEvent.java}    |  14 +-
 .../affinity/event/AffinityEventParameters.java}   |  44 +-
 .../affinity/RendezvousAffinityFunctionTest.java   |   3 +-
 .../ignite/internal/baseline/BaselineManager.java  |   1 +
 modules/metastorage-client/pom.xml                 |   8 +
 ...viceTest.java => ITMetaStorageServiceTest.java} | 751 ++++++++++-----------
 .../internal/metastorage/MetaStorageManager.java   |  40 +-
 modules/network/pom.xml                            |  31 +
 .../ignite/network/ConnectionManagerTest.java      | 184 +++++
 .../ignite/network/DirectSerializationTest.java    | 110 ---
 .../network/TestMessageSerializationFactory.java   |   2 +-
 .../network/scalecube/ITNodeRestartsTest.java      | 142 ++++
 .../scalecube/ITScaleCubeNetworkMessagingTest.java |  32 +-
 .../TestScaleCubeClusterServiceFactory.java        |  40 ++
 .../org/apache/ignite/network/ClusterNode.java     |  41 +-
 .../apache/ignite/network/NetworkException.java    |  33 -
 .../org/apache/ignite/network/TopologyService.java |   9 +
 .../internal/direct/DirectMarshallingUtils.java}   |  34 +-
 .../internal/direct/DirectMessageReader.java       |   2 +-
 .../internal/direct/DirectMessageWriter.java       |  14 +-
 .../stream/DirectByteBufferStreamImplV1.java       |   2 +-
 .../network/internal/netty/ConnectionManager.java  | 250 +++++++
 .../network/internal/netty/InboundDecoder.java     | 125 ++++
 .../network/internal/netty/MessageHandler.java     |  48 ++
 .../ignite/network/internal/netty/NettyClient.java | 162 +++++
 .../ignite/network/internal/netty/NettySender.java | 166 +++++
 .../ignite/network/internal/netty/NettyServer.java | 276 ++++++++
 .../ignite/network/internal/netty/NettyUtils.java  |  77 +++
 .../scalecube/DelegatingTransportFactory.java      |  25 +-
 .../scalecube/ScaleCubeClusterServiceFactory.java  |  31 +-
 .../ScaleCubeDirectMarshallerTransport.java        | 248 +++++++
 .../scalecube/ScaleCubeMessagingService.java       |  26 +-
 .../scalecube/ScaleCubeTopologyService.java        |  53 +-
 .../scalecube/message/ScaleCubeMessage.java        |  62 ++
 .../ScaleCubeMessageSerializationFactory.java}     |  50 +-
 .../ignite/network/internal/AllTypesMessage.java   | 154 +++++
 .../network/internal/AllTypesMessageGenerator.java | 215 ++++++
 .../AllTypesMessageSerializationFactory.java       | 562 +++++++++++++++
 .../ignite/network/internal/TestFieldType.java}    |  16 +-
 .../network/internal/netty/InboundDecoderTest.java | 208 ++++++
 .../network/internal/netty/NettyClientTest.java    | 209 ++++++
 .../network/internal/netty/NettyServerTest.java    | 180 +++++
 .../message/MessageSerializationRegistryTest.java  | 105 +++
 .../src/test/resources/simplelogger.properties     |  52 ++
 .../raft/client/service/RaftGroupServiceTest.java  |   6 +-
 modules/raft/pom.xml                               |   7 +
 .../raft/server/ITRaftCounterServerTest.java       |  15 +-
 .../ignite/raft/server/impl/RaftServerImpl.java    |   2 -
 .../apache/ignite/internal/app/IgnitionImpl.java   |   8 +-
 modules/table/pom.xml                              |  16 +-
 .../distributed/ITDistributedTableTest.java}       |  16 +-
 .../internal/table/distributed/TableManager.java   | 274 ++++----
 parent/pom.xml                                     |   7 +
 55 files changed, 4405 insertions(+), 962 deletions(-)
 copy modules/{table/src/main/java/org/apache/ignite/internal/table/event/TableEvent.java => affinity/src/main/java/org/apache/ignite/internal/affinity/event/AffinityEvent.java} (75%)
 copy modules/{table/src/main/java/org/apache/ignite/internal/table/TableSchemaViewImpl.java => affinity/src/main/java/org/apache/ignite/internal/affinity/event/AffinityEventParameters.java} (52%)
 rename modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/{MetaStorageServiceTest.java => ITMetaStorageServiceTest.java} (55%)
 create mode 100644 modules/network/src/integrationTest/java/org/apache/ignite/network/ConnectionManagerTest.java
 delete mode 100644 modules/network/src/integrationTest/java/org/apache/ignite/network/DirectSerializationTest.java
 create mode 100644 modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/ITNodeRestartsTest.java
 create mode 100644 modules/network/src/integrationTest/java/org/apache/ignite/network/scalecube/TestScaleCubeClusterServiceFactory.java
 delete mode 100644 modules/network/src/main/java/org/apache/ignite/network/NetworkException.java
 copy modules/{metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/MultipleEntryResponse.java => network/src/main/java/org/apache/ignite/network/internal/direct/DirectMarshallingUtils.java} (55%)
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/ConnectionManager.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/InboundDecoder.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/MessageHandler.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyClient.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettySender.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyServer.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/internal/netty/NettyUtils.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/scalecube/ScaleCubeDirectMarshallerTransport.java
 create mode 100644 modules/network/src/main/java/org/apache/ignite/network/scalecube/message/ScaleCubeMessage.java
 copy modules/network/src/{integrationTest/java/org/apache/ignite/network/TestMessageSerializationFactory.java => main/java/org/apache/ignite/network/scalecube/message/ScaleCubeMessageSerializationFactory.java} (61%)
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/AllTypesMessage.java
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/AllTypesMessageGenerator.java
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/AllTypesMessageSerializationFactory.java
 copy modules/{core/src/main/java/org/apache/ignite/internal/tostring/IgniteToStringExclude.java => network/src/test/java/org/apache/ignite/network/internal/TestFieldType.java} (71%)
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/netty/InboundDecoderTest.java
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/netty/NettyClientTest.java
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/internal/netty/NettyServerTest.java
 create mode 100644 modules/network/src/test/java/org/apache/ignite/network/message/MessageSerializationRegistryTest.java
 create mode 100644 modules/network/src/test/resources/simplelogger.properties
 rename modules/table/src/{test/java/org/apache/ignite/table/distributed/DistributedTableTest.java => integrationTest/java/org/apache/ignite/distributed/ITDistributedTableTest.java} (95%)

[ignite-3] 03/09: IGNITE-14389 Added get and do smth semantic

Posted by ag...@apache.org.
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 1a1e8be92d598fb886a07a8aabda78084fb7d23c
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Tue Apr 6 22:25:03 2021 +0300

    IGNITE-14389 Added get and do smth semantic
---
 .../metastorage/server/KeyValueStorage.java        |  12 +-
 .../server/SimpleInMemoryKeyValueStorage.java      | 147 +++++++++----
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 235 +++++++++++++++++++--
 3 files changed, 329 insertions(+), 65 deletions(-)

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
index e245e08..ead1043 100644
--- 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
@@ -11,16 +11,20 @@ public interface KeyValueStorage {
     long updateCounter();
 
     @NotNull
-    Entry put(byte[] key, byte[] value);
-
-    @NotNull
     Entry get(byte[] key);
 
     @NotNull
     Entry get(byte[] key, long rev);
 
+    void put(byte[] key, byte[] value);
+
+    @NotNull
+    Entry getAndPut(byte[] key, byte[] value);
+
+    void remove(byte[] key);
+
     @NotNull
-    Entry remove(byte[] key);
+    Entry getAndRemove(byte[] key);
 
     Iterator<Entry> iterate(byte[] key);
 
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
index b764998..3700f4a 100644
--- 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
@@ -46,49 +46,26 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return updCntr;
     }
 
-    @NotNull
-    @Override public Entry put(byte[] key, byte[] bytes) {
+    @Override public void put(byte[] key, byte[] value) {
         synchronized (mux) {
-            long curRev = ++rev;
-
-            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.
-            NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
-
-            Value val = new Value(bytes, curUpdCntr);
-
-            entries.put(key, val);
+            doPut(key, value);
+        }
+    }
 
-            revsIdx.put(curRev, entries);
+    @NotNull
+    @Override public Entry getAndPut(byte[] key, byte[] bytes) {
+        synchronized (mux) {
+            long lastRev = doPut(key, bytes);
 
             // Return previous value.
-            if (lastRev == 0)
-                return Entry.empty(key);
-
-            NavigableMap<byte[], Value> lastRevVals = revsIdx.get(lastRev);
-
-            Value lastVal = lastRevVals.get(key);
-
-            Entry res = new Entry(key, lastVal.bytes(), lastRev, lastVal.updateCounter());
-
-            //TODO: notify watchers
-
-            return res;
+            return doGetValue(key, lastRev);
         }
     }
 
     @NotNull
     @Override public Entry get(byte[] key) {
         synchronized (mux) {
-            return doGet(key, LATEST_REV);
+            return doGet(key, LATEST_REV, false);
         }
     }
 
@@ -96,19 +73,31 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     @TestOnly
     @Override public Entry get(byte[] key, long rev) {
         synchronized (mux) {
-            return doGet(key, rev);
+            return doGet(key, rev, true);
+        }
+    }
+
+    @Override
+    public void remove(byte[] key) {
+        synchronized (mux) {
+            Entry e = doGet(key, LATEST_REV, false);
+
+            if (e.empty() || e.tombstone())
+                return;
+
+            doPut(key, TOMBSTONE);
         }
     }
 
     @NotNull
-    @Override public Entry remove(byte[] key) {
+    @Override public Entry getAndRemove(byte[] key) {
         synchronized (mux) {
-            Entry e = doGet(key, LATEST_REV);
+            Entry e = doGet(key, LATEST_REV, false);
 
-            if (e.value() == null)
+            if (e.empty() || e.tombstone())
                 return e;
 
-            return put(key, TOMBSTONE);
+            return getAndPut(key, TOMBSTONE);
         }
     }
 
@@ -233,25 +222,91 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
      * @param rev Revision.
      * @return Entry for given key.
      */
-    @NotNull private Entry doGet(byte[] key, long rev) {
+    @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 lrev = rev == LATEST_REV ? lastRevision(revs) : rev;
+        long lastRev;
 
-        NavigableMap<byte[], Value> entries = revsIdx.get(lrev);
+        if (rev == LATEST_REV)
+            lastRev = lastRevision(revs);
+        else
+            lastRev = exactRev ? rev : maxRevision(revs, rev);
 
-        if (entries == null || entries.isEmpty())
+        // lastRev can be -1 if maxRevision return -1.
+        if (lastRev == -1)
             return Entry.empty(key);
 
-        Value val = entries.get(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 = ++rev;
+
+        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.
+        NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+        Value val = new Value(bytes, curUpdCntr);
+
+        entries.put(key, val);
 
-        if (val.tombstone())
-            return Entry.tombstone(key, lrev, val.updateCounter());
+        revsIdx.put(curRev, entries);
 
-        return new Entry(key, val.bytes() , lrev, val.updateCounter());
+        return lastRev;
     }
 
     private static boolean isPrefix(byte[] pref, byte[] term) {
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
index eae76fd..5b797fc 100644
--- 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
@@ -22,7 +22,212 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @Test
-    void putGetRemoveCompact() {
+    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
+    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 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 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);
@@ -34,7 +239,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(0, storage.updateCounter());
 
         // Previous entry is empty.
-        Entry emptyEntry = storage.put(key1, val1_1);
+        Entry emptyEntry = storage.getAndPut(key1, val1_1);
 
         assertEquals(1, storage.revision());
         assertEquals(1, storage.updateCounter());
@@ -53,7 +258,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(1, storage.updateCounter());
 
         // Previous entry is empty.
-        emptyEntry = storage.put(key2, val2_2);
+        emptyEntry = storage.getAndPut(key2, val2_2);
 
         assertEquals(2, storage.revision());
         assertEquals(2, storage.updateCounter());
@@ -72,7 +277,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(2, storage.updateCounter());
 
         // Previous entry is not empty.
-        e1_1 = storage.put(key1, val1_3);
+        e1_1 = storage.getAndPut(key1, val1_3);
 
         assertFalse(e1_1.empty());
         assertFalse(e1_1.tombstone());
@@ -96,7 +301,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(3, storage.updateCounter());
 
         // Remove existing entry.
-        Entry e2_2 = storage.remove(key2);
+        Entry e2_2 = storage.getAndRemove(key2);
 
         assertFalse(e2_2.empty());
         assertFalse(e2_2.tombstone());
@@ -108,7 +313,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(4, storage.updateCounter());
 
         // Remove already removed entry.
-        Entry tombstoneEntry = storage.remove(key2);
+        Entry tombstoneEntry = storage.getAndRemove(key2);
 
         assertFalse(tombstoneEntry.empty());
         assertTrue(tombstoneEntry.tombstone());
@@ -120,11 +325,11 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, storage.revision());
         assertEquals(4, storage.updateCounter());
-        assertTrue(storage.remove(key2).empty());
+        assertTrue(storage.getAndRemove(key2).empty());
         assertTrue(storage.get(key2).empty());
 
         // Remove existing entry.
-        e1_3 = storage.remove(key1);
+        e1_3 = storage.getAndRemove(key1);
 
         assertFalse(e1_3.empty());
         assertFalse(e1_3.tombstone());
@@ -136,7 +341,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(5, storage.updateCounter());
 
         // Remove already removed entry.
-        tombstoneEntry = storage.remove(key1);
+        tombstoneEntry = storage.getAndRemove(key1);
 
         assertFalse(tombstoneEntry.empty());
         assertTrue(tombstoneEntry.tombstone());
@@ -148,12 +353,12 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(5, storage.revision());
         assertEquals(5, storage.updateCounter());
-        assertTrue(storage.remove(key1).empty());
+        assertTrue(storage.getAndRemove(key1).empty());
         assertTrue(storage.get(key1).empty());
     }
 
     @Test
-    void compact() {
+    public void compact() {
         assertEquals(0, storage.revision());
         assertEquals(0, storage.updateCounter());
 
@@ -179,7 +384,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertEquals(6, storage.revision());
         assertEquals(6, storage.updateCounter());
 
-        storage.remove(k(3));
+        storage.getAndRemove(k(3));
 
         assertEquals(7, storage.revision());
         assertEquals(7, storage.updateCounter());
@@ -218,7 +423,7 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @Test
-    void iterate() {
+    public void iterate() {
         TreeMap<String, String> expFooMap = new TreeMap<>();
         TreeMap<String, String> expKeyMap = new TreeMap<>();
         TreeMap<String, String> expZooMap = new TreeMap<>();
@@ -270,13 +475,13 @@ class SimpleInMemoryKeyValueStorageTest {
 
             byte[] val = valStr.getBytes();
 
-            storage.put(key, val);
+            storage.getAndPut(key, val);
         }
     }
 
     private static void fill(KeyValueStorage storage, int keySuffix, int num) {
         for (int i = 0; i < num; i++)
-            storage.put(k(keySuffix), kv(keySuffix, i + 1));
+            storage.getAndPut(k(keySuffix), kv(keySuffix, i + 1));
     }
 
     private static byte[] k(int k) {

[ignite-3] 07/09: IGNITE-14389 Implemented cursor for ranges and watches.

Posted by ag...@apache.org.
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 a0febd6acb00884c2a520a78d1195f689a212538
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Thu Apr 29 02:44:00 2021 +0300

    IGNITE-14389 Implemented cursor for ranges and watches.
---
 .../ignite/internal/metastorage/server/Entry.java  |  17 +
 .../internal/metastorage/server/EntryEvent.java    |  58 +++
 .../metastorage/server/KeyValueStorage.java        |  33 +-
 .../server/SimpleInMemoryKeyValueStorage.java      | 333 +++++++++------
 .../ignite/internal/metastorage/server/Value.java  |  17 +
 .../ignite/internal/metastorage/server/Watch.java  |  45 ---
 .../internal/metastorage/server/WatchEvent.java    |  54 +++
 .../internal/metastorage/server/Watcher.java       |  13 -
 .../internal/metastorage/server/WatcherImpl.java   |  58 ---
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 446 ++++++++++++++++++---
 10 files changed, 764 insertions(+), 310 deletions(-)

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
index 263a88b..87b5471 100644
--- 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
@@ -1,3 +1,20 @@
+/*
+ * 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;
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
new file mode 100644
index 0000000..554a3a7
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+/**
+ * Represent an change event for particular key and entry.
+ */
+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.
+     */
+    EntryEvent(Entry oldEntry, Entry curEntry) {
+        this.oldEntry = oldEntry;
+        this.entry = curEntry;
+    }
+
+    /**
+     * Returns old entry.
+     *
+     * @return Old entry.
+     */
+    public Entry oldEntry() {
+        return oldEntry;
+    }
+
+    /**
+     * Rreturns new entry.
+     *
+     * @return New entry.
+     */
+    public Entry entry() {
+        return entry;
+    }
+}
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
index 526e4fb..5d6da44 100644
--- 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
@@ -1,13 +1,28 @@
-package org.apache.ignite.internal.metastorage.server;
+/*
+ * 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.
+ */
 
-import org.jetbrains.annotations.NotNull;
+package org.apache.ignite.internal.metastorage.server;
 
 import java.util.Collection;
-import java.util.Iterator;
 import java.util.List;
+import org.apache.ignite.metastorage.common.Cursor;
+import org.jetbrains.annotations.NotNull;
 
 public interface KeyValueStorage {
-
     long revision();
 
     long updateCounter();
@@ -44,11 +59,15 @@ public interface KeyValueStorage {
     @NotNull
     Collection<Entry> getAndRemoveAll(List<byte[]> keys);
 
-    Iterator<Entry> range(byte[] keyFrom, byte[] keyTo);
+    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);
 
-    Iterator<Entry> iterate(byte[] key);
+    Cursor<WatchEvent> watch(byte[] key, long rev);
 
-    //Iterator<Entry> iterate(long rev);
+    Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
 
     void compact();
 }
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
index 32f720e..b37c96a 100644
--- 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
@@ -1,8 +1,34 @@
-package org.apache.ignite.internal.metastorage.server;
+/*
+ * 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.
+ */
 
-import java.util.*;
-import java.util.function.Consumer;
+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.metastorage.common.Cursor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.TestOnly;
@@ -13,15 +39,11 @@ import static org.apache.ignite.internal.metastorage.server.Value.TOMBSTONE;
  * WARNING: Only for test purposes and only for non-distributed (one static instance) storage.
  */
 public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
-    private static final Comparator<byte[]> LEXICOGRAPHIC_COMPARATOR = Arrays::compare;
+    private static final Comparator<byte[]> CMP = Arrays::compare;
 
     private static final long LATEST_REV = -1;
 
-    private final Watcher watcher;
-
-    private final List<Cursor<Entry>> rangeCursors = new ArrayList<>();
-
-    private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+    private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(CMP);
 
     private NavigableMap<Long, NavigableMap<byte[], Value>> revsIdx = new TreeMap<>();
 
@@ -31,10 +53,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
     private final Object mux = new Object();
 
-    public SimpleInMemoryKeyValueStorage(Watcher watcher) {
-        this.watcher = watcher;
-    }
-
     @Override public long revision() {
         return rev;
     }
@@ -141,9 +159,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
             List<byte[]> vals = new ArrayList<>(keys.size());
 
-            for (int i = 0; i < keys.size(); i++) {
-                byte[] key = keys.get(i);
-
+            for (byte[] key : keys) {
                 Entry e = doGet(key, LATEST_REV, false);
 
                 if (e.empty() || e.tombstone())
@@ -169,9 +185,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
             List<byte[]> vals = new ArrayList<>(keys.size());
 
-            for (int i = 0; i < keys.size(); i++) {
-                byte[] key = keys.get(i);
-
+            for (byte[] key : keys) {
                 Entry e = doGet(key, LATEST_REV, false);
 
                 res.add(e);
@@ -190,90 +204,45 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return res;
     }
 
-    @Override
-    public Iterator<Entry> range(byte[] keyFrom, byte[] keyTo) {
-        return null;
+    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo) {
+        return new RangeCursor(keyFrom, keyTo, rev);
     }
 
-    @Override public Iterator<Entry> iterate(byte[] keyFrom) {
-        synchronized (mux) {
-            NavigableMap<byte[], List<Long>> tailMap = keysIdx.tailMap(keyFrom, true);
-
-            final Iterator<Map.Entry<byte[], List<Long>>> it = tailMap.entrySet().iterator();
-
-            return new Iterator<>() {
-                private Map.Entry<byte[], List<Long>> curr;
-                private boolean hasNext;
-
-                private void advance() {
-                    if (it.hasNext()) {
-                        Map.Entry<byte[], List<Long>> e = it.next();
-
-                        byte[] key = e.getKey();
-
-                        if (!isPrefix(keyFrom, key))
-                            hasNext = false;
-                        else {
-                            curr = e;
-
-                            hasNext = true;
-                        }
-                    } else
-                        hasNext = false;
-                }
-
-                @Override
-                public boolean hasNext() {
-                    synchronized (mux) {
-                        if (curr == null)
-                            advance();
-
-                        return hasNext;
-                    }
-                }
-
-                @Override
-                public Entry next() {
-                    synchronized (mux) {
-                        if (!hasNext())
-                            throw new NoSuchElementException();
-
-                        Map.Entry<byte[], List<Long>> e = curr;
-
-                        curr = null;
+    @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo, long revUpperBound) {
+        return new RangeCursor(keyFrom, keyTo, revUpperBound);
+    }
 
-                        byte[] key = e.getKey();
+    @Override public Cursor<WatchEvent> watch(byte[] keyFrom, byte[] keyTo, long rev) {
+        assert keyFrom != null;
+        assert rev > 0;
 
-                        List<Long> revs = e.getValue();
+        return new WatchCursor(rev, k ->
+            CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
+        );
+    }
 
-                        long rev = revs == null || revs.isEmpty() ? 0 : lastRevision(revs);
+    @Override public Cursor<WatchEvent> watch(byte[] key, long rev) {
+        assert key != null;
+        assert rev > 0;
 
-                        if (rev == 0) {
-                            throw new IllegalStateException("rev == 0");
-                            //return new AbstractMap.SimpleImmutableEntry<>(key, null);
-                        }
+        return new WatchCursor(rev, k -> CMP.compare(k, key) == 0);
+    }
 
-                        NavigableMap<byte[], Value> vals = revsIdx.get(rev);
+    @Override public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+        assert keys != null && !keys.isEmpty();
+        assert rev > 0;
 
-                        if (vals == null || vals.isEmpty()) {
-                            throw new IllegalStateException("vals == null || vals.isEmpty()");
-                            //return new AbstractMap.SimpleImmutableEntry<>(key, null);
-                        }
+        TreeSet<byte[]> keySet = new TreeSet<>(CMP);
 
-                        Value val = vals.get(key);
+        keySet.addAll(keys);
 
-                        return val.tombstone() ?
-                                Entry.tombstone(key, rev, val.updateCounter()) :
-                                new Entry(key, val.bytes(), rev, val.updateCounter());
-                    }
-                }
-            };
-        }
+        return new WatchCursor(rev, keySet::contains);
     }
 
+
     @Override public void compact() {
         synchronized (mux) {
-            NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+            NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(CMP);
 
             NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new TreeMap<>();
 
@@ -302,7 +271,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
             NavigableMap<byte[], Value> compactedKv = compactedRevsIdx.computeIfAbsent(
                     lastRev,
-                    k -> new TreeMap<>(LEXICOGRAPHIC_COMPARATOR)
+                    k -> new TreeMap<>(CMP)
             );
 
             compactedKv.put(key, lastVal);
@@ -409,7 +378,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         revs.add(curRev);
 
         // Update revsIdx.
-        NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+        NavigableMap<byte[], Value> entries = new TreeMap<>(CMP);
 
         Value val = new Value(bytes, curUpdCntr);
 
@@ -423,7 +392,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     private long doPutAll(long curRev, List<byte[]> keys, List<byte[]> bytesList) {
         synchronized (mux) {
             // Update revsIdx.
-            NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+            NavigableMap<byte[], Value> entries = new TreeMap<>(CMP);
 
             for (int i = 0; i < keys.size(); i++) {
                 byte[] key = keys.get(i);
@@ -452,19 +421,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         }
     }
 
-
-    private static boolean isPrefix(byte[] pref, byte[] term) {
-        if (pref.length > term.length)
-            return false;
-
-        for (int i = 0; i < pref.length - 1; i++) {
-            if (pref[i] != term[i])
-                return false;
-        }
-
-        return true;
-    }
-
     private static long lastRevision(List<Long> revs) {
         return revs.get(revs.size() - 1);
     }
@@ -481,55 +437,184 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         private final byte[] keyFrom;
         private final byte[] keyTo;
         private final long rev;
-        private byte[] curKey;
+        private Entry nextRetEntry;
+        private byte[] lastRetKey;
+        private boolean finished;
 
-        public RangeCursor(byte[] keyFrom, byte[] keyTo, long rev) {
+        RangeCursor(byte[] keyFrom, byte[] keyTo, long rev) {
             this.keyFrom = keyFrom;
             this.keyTo = keyTo;
             this.rev = rev;
         }
 
         @Override public void close() throws Exception {
-
+            // TODO: implement.
         }
 
         @NotNull
         @Override public Iterator<Entry> iterator() {
-            return new Iterator<Entry>() {
+            return new Iterator<>() {
                 @Override public boolean hasNext() {
                     synchronized (mux) {
-                        byte[] key = keysIdx.ceilingKey(curKey);
+                        while (true) {
+                            if (finished)
+                                return false;
+
+                            if (nextRetEntry != null)
+                                return true;
 
-                        return key != null;
+                            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;
+                            }
+                        }
                     }
                 }
 
                 @Override public Entry next() {
                     synchronized (mux) {
-                        Map.Entry<byte[], List<Long>> e = keysIdx.ceilingEntry(curKey);
-
-                        if (e == null)
-                            throw new NoSuchElementException();
+                        while (true) {
+                            if (finished)
+                                throw new NoSuchElementException();
 
-                        List<Long> revs = e.getValue();
+                            if (nextRetEntry != null) {
+                                Entry e = nextRetEntry;
 
-                        assert revs != null && !revs.isEmpty() :
-                                "Revisions should not be empty: [revs=" + revs + ']';
+                                nextRetEntry = null;
 
-                        //lastRevision(re)
+                                lastRetKey = e.key();
 
-                        return null;
+                                return e;
+                            } else
+                                hasNext();
+                        }
                     }
                 }
             };
         }
+    }
 
-        @Override public void forEach(Consumer<? super Entry> action) {
-            Cursor.super.forEach(action);
+    private class WatchCursor implements Cursor<WatchEvent> {
+        private final Predicate<byte[]> p;
+        private long lastRetRev;
+        private long nextRetRev = -1;
+
+        WatchCursor(long rev, Predicate<byte[]> p) {
+            this.p = p;
+            this.lastRetRev = rev - 1;
         }
 
-        @Override public Spliterator<Entry> spliterator() {
-            return Cursor.super.spliterator();
+        @Override public void close() throws Exception {
+            // TODO: implement
+        }
+
+        @NotNull
+        @Override public Iterator<WatchEvent> iterator() {
+            return new Iterator<>() {
+                @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++;
+                        }
+                    }
+                }
+
+                @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-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
index 250a5ea..a438fd4 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
@@ -1,3 +1,20 @@
+/*
+ * 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;
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
deleted file mode 100644
index 26cfa5c..0000000
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
+++ /dev/null
@@ -1,45 +0,0 @@
-package org.apache.ignite.internal.metastorage.server;
-
-import org.jetbrains.annotations.Nullable;
-
-import java.util.Arrays;
-import java.util.Comparator;
-
-public class Watch {
-    private static final Comparator<byte[]> CMP = Arrays::compare;
-
-    private static final long ANY_REVISION = -1;
-
-    @Nullable
-    private byte[] startKey;
-
-    @Nullable
-    private byte[] endKey;
-
-    long rev = ANY_REVISION;
-
-    public void startKey(byte[] startKey) {
-        this.startKey = startKey;
-    }
-
-    public void endKey(byte[] endKey) {
-        this.endKey = endKey;
-    }
-
-    public void revision(long rev) {
-        this.rev = rev;
-    }
-
-    public void notify(Entry e) {
-        if (startKey != null && CMP.compare(e.key(), startKey) < 0)
-            return;
-
-        if (endKey != null && CMP.compare(e.key(), endKey) > 0)
-            return;
-
-        if (rev != ANY_REVISION && e.revision() <= rev)
-            return;
-
-        System.out.println("Entry: key=" + new String(e.key()) + ", value=" + new String(e.value()) + ", rev=" + e.revision());
-    }
-}
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..561f203
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+public class WatchEvent {
+    private final List<EntryEvent> entryEvts;
+
+    private final boolean batch;
+
+    /**
+     * 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.batch = entryEvts.size() > 1;
+        this.entryEvts = entryEvts;
+    }
+
+    public boolean batch() {
+        return batch;
+    }
+
+    public Collection<EntryEvent> entryEvents() {
+        return entryEvts;
+    }
+
+    public EntryEvent entryEvent() {
+        if (batch)
+            throw new IllegalStateException("Watch event represents a batch of events.");
+
+        return entryEvts.get(0);
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java
deleted file mode 100644
index 5516d06..0000000
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.metastorage.server;
-
-import org.jetbrains.annotations.NotNull;
-
-public interface Watcher {
-    void register(@NotNull Watch watch);
-
-    void notify(@NotNull Entry e);
-
-    //TODO: implement
-    void cancel(@NotNull Watch watch);
-}
-
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java
deleted file mode 100644
index dc126a0..0000000
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package org.apache.ignite.internal.metastorage.server;
-
-import org.jetbrains.annotations.NotNull;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-public class WatcherImpl implements Watcher {
-    private final BlockingQueue<Entry> queue = new LinkedBlockingQueue<>();
-
-    private final List<Watch> watches = new ArrayList<>();
-
-    private volatile boolean stop;
-
-    private final Object mux = new Object();
-
-    @Override public void register(@NotNull Watch watch) {
-        synchronized (mux) {
-            watches.add(watch);
-        }
-    }
-
-    @Override public void notify(@NotNull Entry e) {
-        queue.offer(e);
-    }
-
-    @Override
-    public void cancel(@NotNull Watch watch) {
-        throw new UnsupportedOperationException("Not implemented yet.");
-    }
-
-    public void shutdown() {
-        stop = true;
-    }
-
-    private class WatcherWorker implements Runnable {
-        @Override public void run() {
-            while (!stop) {
-                try {
-                    Entry e = queue.poll(100, TimeUnit.MILLISECONDS);
-
-                    if (e != null) {
-                        synchronized (mux) {
-                            watches.forEach(w -> w.notify(e));
-                        }
-                    }
-                }
-                catch (InterruptedException interruptedException) {
-                    // No-op.
-                }
-            }
-        }
-    }
-}
-
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
index 4a73137..27df790 100644
--- 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
@@ -1,17 +1,34 @@
+/*
+ * 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.TreeMap;
-import java.util.function.Function;
+import java.util.NoSuchElementException;
 import java.util.stream.Collectors;
+import org.apache.ignite.metastorage.common.Cursor;
 import org.apache.ignite.metastorage.common.Key;
-import org.jetbrains.annotations.NotNull;
 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.*;
 
 class SimpleInMemoryKeyValueStorageTest {
@@ -19,7 +36,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
     @BeforeEach
     public void setUp() {
-        storage = new SimpleInMemoryKeyValueStorage(new NoOpWatcher());
+        storage = new SimpleInMemoryKeyValueStorage();
     }
 
     @Test
@@ -91,7 +108,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -166,7 +183,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -204,7 +221,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         e1 = map.get(new Key(key1));
@@ -308,7 +325,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -382,7 +399,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(3, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -417,7 +434,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         e1 = map.get(new Key(key1));
@@ -601,7 +618,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -676,7 +693,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         Entry e1 = map.get(new Key(key1));
@@ -719,7 +736,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
 
         // Test regular put value.
         e1 = map.get(new Key(key1));
@@ -987,60 +1004,377 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @Test
-    public void iterate() {
-        TreeMap<String, String> expFooMap = new TreeMap<>();
-        TreeMap<String, String> expKeyMap = new TreeMap<>();
-        TreeMap<String, String> expZooMap = new TreeMap<>();
+    public void rangeCursor() {
+        byte[] key1 = k(1);
+        byte[] val1 = kv(1, 1);
 
-        fill("foo", storage, expFooMap);
-        fill("key", storage, expKeyMap);
-        fill("zoo", storage, expZooMap);
+        byte[] key2 = k(2);
+        byte[] val2 = kv(2, 2);
 
-        assertEquals(300, storage.revision());
-        assertEquals(300, storage.updateCounter());
+        byte[] key3 = k(3);
+        byte[] val3 = kv(3, 3);
 
-        assertIterate("key", storage, expKeyMap);
-        assertIterate("zoo", storage, expZooMap);
-        assertIterate("foo", storage, expFooMap);
-    }
 
-    private void assertIterate(String pref,  KeyValueStorage storage, TreeMap<String, String> expMap) {
-        Iterator<Entry> it = storage.iterate((pref + "_").getBytes());
-        Iterator<Map.Entry<String, String>> expIt = expMap.entrySet().iterator();
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        storage.putAll(List.of(key1, key2, key3), List.of(val1, val2, val3));
+
+        assertEquals(1, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        // Range for latest revision without max bound.
+        Cursor<Entry> cur = storage.range(key1, null);
+
+        Iterator<Entry> it = cur.iterator();
+
+        assertTrue(it.hasNext());
+
+        Entry e1 = it.next();
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertArrayEquals(key1, e1.key());
+        assertArrayEquals(val1, e1.value());
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
+
+        assertTrue(it.hasNext());
+
+        Entry e2 = it.next();
 
-        // Order.
-        while (it.hasNext()) {
-            Entry entry = it.next();
-            Map.Entry<String, String> expEntry = expIt.next();
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertArrayEquals(key2, e2.key());
+        assertArrayEquals(val2, e2.value());
+        assertEquals(1, e2.revision());
+        assertEquals(2, e2.updateCounter());
+
+        // Deliberately don't call it.hasNext()
+
+        Entry e3 = it.next();
+
+        assertFalse(e3.empty());
+        assertFalse(e3.tombstone());
+        assertArrayEquals(key3, e3.key());
+        assertArrayEquals(val3, e3.value());
+        assertEquals(1, e3.revision());
+        assertEquals(3, e3.updateCounter());
+
+        assertFalse(it.hasNext());
 
-            assertEquals(expEntry.getKey(), new String(entry.key()));
-            assertEquals(expEntry.getValue(), new String(entry.value()));
+        try {
+            it.next();
+
+            fail();
+        }
+        catch (NoSuchElementException e) {
+            System.out.println();
+            // No-op.
         }
 
-        // Range boundaries.
-        it = storage.iterate((pref + '_').getBytes());
+        // Range for latest revision with max bound.
+        cur = storage.range(key1, key3);
+
+        it = cur.iterator();
+
+        assertTrue(it.hasNext());
+
+        e1 = it.next();
+
+        assertFalse(e1.empty());
+        assertFalse(e1.tombstone());
+        assertArrayEquals(key1, e1.key());
+        assertArrayEquals(val1, e1.value());
+        assertEquals(1, e1.revision());
+        assertEquals(1, e1.updateCounter());
 
-        while (it.hasNext()) {
-            Entry entry = it.next();
+        assertTrue(it.hasNext());
 
-            assertTrue(expMap.containsKey(new String(entry.key())));
+        e2 = it.next();
+
+        assertFalse(e2.empty());
+        assertFalse(e2.tombstone());
+        assertArrayEquals(key2, e2.key());
+        assertArrayEquals(val2, e2.value());
+        assertEquals(1, e2.revision());
+        assertEquals(2, e2.updateCounter());
+
+        assertFalse(it.hasNext());
+
+        try {
+            it.next();
+
+            fail();
+        }
+        catch (NoSuchElementException e) {
+            System.out.println();
+            // No-op.
         }
     }
 
-    private static void fill(String pref, KeyValueStorage storage, TreeMap<String, String> expMap) {
-        for (int i = 0; i < 100; i++) {
-            String keyStr = pref + '_' + i;
+    @Test
+    public void watchCursorForRange() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
 
-            String valStr = "val_" + i;
+        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);
 
-            expMap.put(keyStr, valStr);
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
 
-            byte[] key = keyStr.getBytes();
+        // Watch for all updates starting from revision 2.
+        Cursor<WatchEvent> cur = storage.watch(key1, null, 2);
 
-            byte[] val = valStr.getBytes();
+        Iterator<WatchEvent> it = cur.iterator();
 
-            storage.getAndPut(key, val);
-        }
+        assertFalse(it.hasNext());
+        assertNull(it.next());
+
+        storage.putAll(List.of(key1, key2), List.of(val1_1, val2_1));
+
+        assertEquals(1, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        // Revision is less than 2.
+        assertFalse(it.hasNext());
+        assertNull(it.next());
+
+        storage.putAll(List.of(key2, key3), List.of(val2_2, val3_1));
+
+        assertEquals(2, storage.revision());
+        assertEquals(4, storage.updateCounter());
+
+        // Revision is 2.
+        assertTrue(it.hasNext());
+
+        WatchEvent watchEvent = it.next();
+
+        assertTrue(watchEvent.batch());
+
+        Map<Key, EntryEvent> map = watchEvent.entryEvents().stream()
+                .collect(Collectors.toMap(evt -> new Key(evt.entry().key()), identity()));
+
+        assertEquals(2, map.size());
+
+        // First update under revision.
+        EntryEvent e2 = map.get(new Key(key2));
+
+        assertNotNull(e2);
+
+        Entry oldEntry2 = e2.oldEntry();
+
+        assertFalse(oldEntry2.empty());
+        assertFalse(oldEntry2.tombstone());
+        assertEquals(1, oldEntry2.revision());
+        assertEquals(2, oldEntry2.updateCounter());
+        assertArrayEquals(key2, oldEntry2.key());
+        assertArrayEquals(val2_1, oldEntry2.value());
+
+        Entry newEntry2 = e2.entry();
+
+        assertFalse(newEntry2.empty());
+        assertFalse(newEntry2.tombstone());
+        assertEquals(2, newEntry2.revision());
+        assertEquals(3, newEntry2.updateCounter());
+        assertArrayEquals(key2, newEntry2.key());
+        assertArrayEquals(val2_2, newEntry2.value());
+
+        // Second update under revision.
+        EntryEvent e3 = map.get(new Key(key3));
+
+        assertNotNull(e3);
+
+        Entry oldEntry3 = e3.oldEntry();
+
+        assertTrue(oldEntry3.empty());
+        assertFalse(oldEntry3.tombstone());
+        assertArrayEquals(key3, oldEntry3.key());
+
+        Entry newEntry3 = e3.entry();
+
+        assertFalse(newEntry3.empty());
+        assertFalse(newEntry3.tombstone());
+        assertEquals(2, newEntry3.revision());
+        assertEquals(4, newEntry3.updateCounter());
+        assertArrayEquals(key3, newEntry3.key());
+        assertArrayEquals(val3_1, newEntry3.value());
+
+        assertFalse(it.hasNext());
+
+        storage.remove(key1);
+
+        assertTrue(it.hasNext());
+
+        watchEvent = it.next();
+
+        assertFalse(watchEvent.batch());
+
+        EntryEvent e1 = watchEvent.entryEvent();
+
+        Entry oldEntry1 = e1.oldEntry();
+
+        assertFalse(oldEntry1.empty());
+        assertFalse(oldEntry1.tombstone());
+        assertEquals(1, oldEntry1.revision());
+        assertEquals(1, oldEntry1.updateCounter());
+        assertArrayEquals(key1, oldEntry1.key());
+        assertArrayEquals(val1_1, oldEntry1.value());
+
+        Entry newEntry1 = e1.entry();
+
+        assertFalse(newEntry1.empty());
+        assertTrue(newEntry1.tombstone());
+        assertEquals(3, newEntry1.revision());
+        assertEquals(5, newEntry1.updateCounter());
+        assertArrayEquals(key1, newEntry1.key());
+        assertNull(newEntry1.value());
+
+        assertFalse(it.hasNext());
+    }
+
+
+    @Test
+    public void watchCursorForKey() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        byte[] key2 = k(2);
+        byte[] val2_1 = kv(2, 21);
+        byte[] val2_2 = kv(2, 22);
+
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        Cursor<WatchEvent> cur = storage.watch(key1, 1);
+
+        Iterator<WatchEvent> it = cur.iterator();
+
+        assertFalse(it.hasNext());
+        assertNull(it.next());
+
+        storage.putAll(List.of(key1, key2), List.of(val1_1, val2_1));
+
+        assertEquals(1, storage.revision());
+        assertEquals(2, storage.updateCounter());
+
+        assertTrue(it.hasNext());
+
+        WatchEvent watchEvent = it.next();
+
+        assertFalse(watchEvent.batch());
+
+        EntryEvent e1 = watchEvent.entryEvent();
+
+        Entry oldEntry1 = e1.oldEntry();
+
+        assertTrue(oldEntry1.empty());
+        assertFalse(oldEntry1.tombstone());
+
+        Entry newEntry1 = e1.entry();
+
+        assertFalse(newEntry1.empty());
+        assertFalse(newEntry1.tombstone());
+        assertEquals(1, newEntry1.revision());
+        assertEquals(1, newEntry1.updateCounter());
+        assertArrayEquals(key1, newEntry1.key());
+        assertArrayEquals(val1_1, newEntry1.value());
+
+        assertFalse(it.hasNext());
+
+        storage.put(key2, val2_2);
+
+        assertFalse(it.hasNext());
+
+        storage.put(key1, val1_2);
+
+        assertTrue(it.hasNext());
+
+        watchEvent = it.next();
+
+        assertFalse(watchEvent.batch());
+
+        e1 = watchEvent.entryEvent();
+
+        oldEntry1 = e1.oldEntry();
+
+        assertFalse(oldEntry1.empty());
+        assertFalse(oldEntry1.tombstone());
+        assertEquals(1, oldEntry1.revision());
+        assertEquals(1, oldEntry1.updateCounter());
+        assertArrayEquals(key1, newEntry1.key());
+        assertArrayEquals(val1_1, newEntry1.value());
+
+         newEntry1 = e1.entry();
+
+        assertFalse(newEntry1.empty());
+        assertFalse(newEntry1.tombstone());
+        assertEquals(3, newEntry1.revision());
+        assertEquals(4, newEntry1.updateCounter());
+        assertArrayEquals(key1, newEntry1.key());
+        assertArrayEquals(val1_2, newEntry1.value());
+
+        assertFalse(it.hasNext());
+    }
+
+    @Test
+    public void watchCursorForKeys() {
+        byte[] key1 = k(1);
+        byte[] val1_1 = kv(1, 11);
+        byte[] val1_2 = kv(1, 12);
+
+        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);
+
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        Cursor<WatchEvent> cur = storage.watch(List.of(key1, key2), 1);
+
+        Iterator<WatchEvent> it = cur.iterator();
+
+        assertFalse(it.hasNext());
+        assertNull(it.next());
+
+        storage.putAll(List.of(key1, key2, key3), List.of(val1_1, val2_1, val3_1));
+
+        assertEquals(1, storage.revision());
+        assertEquals(3, storage.updateCounter());
+
+        assertTrue(it.hasNext());
+
+        WatchEvent watchEvent = it.next();
+
+        assertTrue(watchEvent.batch());
+
+        assertFalse(it.hasNext());
+
+        storage.put(key2, val2_2);
+
+        assertTrue(it.hasNext());
+
+        watchEvent = it.next();
+
+        assertFalse(watchEvent.batch());
+
+        assertFalse(it.hasNext());
+
+        storage.put(key3, val3_2);
+
+        assertFalse(it.hasNext());
     }
 
     private static void fill(KeyValueStorage storage, int keySuffix, int num) {
@@ -1055,18 +1389,4 @@ class SimpleInMemoryKeyValueStorageTest {
     private static byte[] kv(int k, int v) {
         return ("key" + k + '_' + "val" + v).getBytes();
     }
-
-    private static class NoOpWatcher implements Watcher {
-        @Override public void register(@NotNull Watch watch) {
-            // No-op.
-        }
-
-        @Override public void notify(@NotNull Entry e) {
-            // No-op.
-        }
-
-        @Override public void cancel(@NotNull Watch watch) {
-            // No-op.
-        }
-    }
 }
\ No newline at end of file

[ignite-3] 08/09: WIP

Posted by ag...@apache.org.
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 67128be9b0c86835d98d52b597587032c4d873e1
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Wed May 5 18:25:00 2021 +0300

    WIP
---
 .../ignite/internal/affinity/AffinityManager.java  |  34 +-
 .../org/apache/ignite/internal/util}/Cursor.java   |  10 +-
 .../java/org/apache/ignite/lang/ByteArray.java     |  16 +-
 .../client/ITMetaStorageServiceTest.java           | 899 ++++++++++++---------
 .../internal/metastorage/client/CursorImpl.java    |  75 +-
 .../metastorage/client/MetaStorageServiceImpl.java |  83 +-
 .../metastorage/client}/CompactedException.java    |   2 +-
 .../ignite/metastorage/client}/Condition.java      | 228 +++---
 .../ignite/metastorage/client}/Conditions.java     |  48 +-
 .../apache/ignite/metastorage/client}/Entry.java   |   5 +-
 .../ignite/metastorage/client/EntryEvent.java}     |  24 +-
 .../org/apache/ignite/metastorage/client}/Key.java |   2 +-
 .../metastorage/client/MetaStorageService.java     | 105 ++-
 .../ignite/metastorage/client}/Operation.java      |  47 +-
 .../client}/OperationTimeoutException.java         |   2 +-
 .../ignite/metastorage/client}/Operations.java     |  14 +-
 .../ignite/metastorage/client}/WatchEvent.java     |  19 +-
 .../ignite/metastorage/client}/WatchListener.java  |   6 +-
 .../internal/metastorage/common/DummyEntry.java    |  23 +-
 .../metastorage/common/command/GetAllCommand.java  |  13 +-
 .../common/command/GetAndPutAllCommand.java        |  11 +-
 .../common/command/GetAndPutCommand.java           |  11 +-
 .../common/command/GetAndRemoveAllCommand.java     |  11 +-
 .../common/command/GetAndRemoveCommand.java        |  11 +-
 .../metastorage/common/command/GetCommand.java     |  18 +-
 .../metastorage/common/command/PutAllCommand.java  |  11 +-
 .../metastorage/common/command/PutCommand.java     |  11 +-
 .../metastorage/common/command/RangeCommand.java   |  25 +-
 .../common/command/RemoveAllCommand.java           |  11 +-
 .../metastorage/common/command/RemoveCommand.java  |  11 +-
 .../common/command/WatchExactKeysCommand.java      |  11 +-
 .../common/command/WatchRangeKeysCommand.java      |  25 +-
 .../common/command/cursor/CursorCloseCommand.java  |   3 +-
 .../command/cursor/CursorHasNextCommand.java       |   3 +-
 .../common/command/cursor/CursorNextCommand.java   |   3 +-
 .../ignite/metastorage/common/KeyValueStorage.java |  91 ---
 .../metastorage/common/KeyValueStorageImpl.java    | 183 -----
 modules/metastorage-server/pom.xml                 |   6 +-
 .../metastorage/server/AbstractCondition.java}     |  27 +-
 .../server/{EntryEvent.java => Condition.java}     |  44 +-
 .../internal/metastorage/server/EntryEvent.java    |   2 +-
 .../metastorage/server/ExistenceCondition.java     |  74 ++
 .../metastorage/server/KeyValueStorage.java        |   4 +-
 .../internal/metastorage/server/Operation.java     | 111 +++
 .../metastorage/server/RevisionCondition.java      | 110 +++
 .../server/SimpleInMemoryKeyValueStorage.java      | 136 +++-
 .../metastorage/server/ValueCondition.java         |  82 ++
 .../internal/metastorage/server/WatchEvent.java    |  11 +-
 .../server}/raft/MetaStorageCommandListener.java   |  36 +-
 .../metastorage/server/ExistenceConditionTest.java |  71 ++
 .../metastorage/server/RevisionConditionTest.java  | 113 +++
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 668 +++++++++++++--
 .../metastorage/server/ValueConditionTest.java     |  60 ++
 .../internal/metastorage/MetaStorageManager.java   | 164 ++--
 .../metastorage/watch/AggregatedWatch.java         |   2 +-
 .../internal/metastorage/watch/KeyCriterion.java   |  44 +-
 .../metastorage/watch/WatchAggregator.java         |  49 +-
 .../internal/metastorage/WatchAggregatorTest.java  |  91 ++-
 .../apache/ignite/internal/app/IgnitionImpl.java   |   5 +-
 .../storage/DistributedConfigurationStorage.java   |  52 +-
 .../internal/table/distributed/TableManager.java   |  19 +-
 61 files changed, 2616 insertions(+), 1470 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%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Cursor.java
rename 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..9f6013a 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,16 @@ import java.util.Arrays;
 import org.jetbrains.annotations.NotNull;
 
 /**
- * A class for handling byte array.
+ * A class wraps {@code byte[]} which provides {@code equals}, {@code hashCode} and lexicographical comparison.
  */
 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 +41,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/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..8601ffd 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,39 +18,30 @@
 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.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.lang.ByteArray;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.metastorage.client.*;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageCommandListener;
 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.message.ScaleCubeMessage;
-import org.apache.ignite.network.scalecube.message.ScaleCubeMessageSerializationFactory;
+import org.apache.ignite.network.scalecube.ScaleCubeClusterServiceFactory;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.message.RaftClientMessageFactory;
 import org.apache.ignite.raft.client.message.impl.RaftClientMessageFactoryImpl;
@@ -63,7 +54,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;
@@ -78,9 +68,9 @@ import static org.mockito.Mockito.verify;
  * Meta storage client tests.
  */
 @SuppressWarnings("WeakerAccess")
-public class ITMetaStorageServiceTest {
+public class MetaStorageServiceTest {
     /** The logger. */
-    private static final IgniteLogger LOG = IgniteLogger.forClass(ITMetaStorageServiceTest.class);
+    private static final IgniteLogger LOG = IgniteLogger.forClass(MetaStorageServiceTest.class);
 
     /** Base network port. */
     private static final int NODE_PORT_BASE = 20_000;
@@ -98,52 +88,71 @@ public class ITMetaStorageServiceTest {
     private static RaftClientMessageFactory FACTORY = new RaftClientMessageFactoryImpl();
 
     /** Network factory. */
-    private static final ClusterServiceFactory NETWORK_FACTORY = new TestScaleCubeClusterServiceFactory();
+    private static final ClusterServiceFactory NETWORK_FACTORY = new ScaleCubeClusterServiceFactory();
 
     /** */
-    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
-        );
+    // TODO: IGNITE-14088 Uncomment and use real serializer provider
+    private static final MessageSerializationRegistry SERIALIZATION_REGISTRY = new MessageSerializationRegistry();
+
+    /**  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 +162,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,39 +189,39 @@ 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()
         );
     }
 
@@ -224,16 +233,13 @@ public class ITMetaStorageServiceTest {
     @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());
     }
 
     /**
@@ -244,44 +250,44 @@ public class ITMetaStorageServiceTest {
     @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 +296,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 +321,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 +361,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 +711,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 +757,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 +783,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 +906,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 +950,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 +968,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 +1019,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 +1027,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 +1062,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 +1095,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 +1107,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 +1119,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 +1131,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 +1139,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 +1149,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..dc76a7c 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
@@ -23,10 +23,10 @@ import java.util.concurrent.ExecutionException;
 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,10 +39,12 @@ 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;
 
     /**
      * @param metaStorageRaftGrpSvc Meta storage raft group service.
@@ -51,37 +53,12 @@ public class CursorImpl<T> implements Cursor<T> {
     CursorImpl(RaftGroupService metaStorageRaftGrpSvc, CompletableFuture<IgniteUuid> initOp) {
         this.metaStorageRaftGrpSvc = metaStorageRaftGrpSvc;
         this.initOp = initOp;
+        this.it = new InnerIterator();
     }
 
     /** {@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 +73,42 @@ public class CursorImpl<T> implements Cursor<T> {
             throw new IgniteInternalException(e);
         }
     }
+
+    @Override
+    public boolean hasNext() {
+        return it.hasNext();
+    }
+
+    @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.<T>run(new CursorNextCommand(cursorId))).get();
+            }
+            catch (InterruptedException | ExecutionException e) {
+                LOG.error("Unable to evaluate cursor hasNext command", e);
+
+                throw new IgniteInternalException(e);
+            }
+        }
+    }
 }
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..fbd1c17 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
@@ -39,17 +39,17 @@ 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.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.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.Condition;
+import org.apache.ignite.metastorage.client.Entry;
+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,45 +77,45 @@ public class MetaStorageServiceImpl implements MetaStorageService {
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> get(@NotNull Key key) {
+    @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key) {
         return metaStorageRaftGrpSvc.run(new GetCommand(key));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> get(@NotNull Key key, long revUpperBound) {
+    @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound) {
         return metaStorageRaftGrpSvc.run(new GetCommand(key, revUpperBound));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys) {
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys) {
         return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys)).
             thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys, long revUpperBound) {
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys, long revUpperBound) {
         return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys, revUpperBound)).
             thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
     }
 
     /** {@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) {
+    @Override public @NotNull CompletableFuture<Entry> getAndPut(@NotNull ByteArray key, @NotNull byte[] value) {
         return metaStorageRaftGrpSvc.run(new GetAndPutCommand(key, value));
     }
 
     /** {@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<>();
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndPutAll(@NotNull Map<ByteArray, byte[]> vals) {
+        List<ByteArray> keys = new ArrayList<>();
         List<byte[]> values = new ArrayList<>();
 
         vals.forEach((key, value) -> {
@@ -128,42 +128,41 @@ public class MetaStorageServiceImpl implements MetaStorageService {
     }
 
     /** {@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) {
+    @Override public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull ByteArray key) {
         return metaStorageRaftGrpSvc.run(new GetAndRemoveCommand(key));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<Key> keys) {
+    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys) {
         return metaStorageRaftGrpSvc.run(new RemoveAllCommand(keys));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<Key, Entry>> getAndRemoveAll(@NotNull Collection<Key> keys) {
+    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Collection<ByteArray> keys) {
         return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAndRemoveAllCommand(keys)).
             thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
     }
 
-    // TODO: IGNITE-14389 Implement.
-    /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Boolean> invoke(@NotNull Condition condition,
-        @NotNull Collection<Operation> success, @NotNull Collection<Operation> failure) {
+    //TODO: implement
+    @Override
+    public @NotNull CompletableFuture<Boolean> invoke(@NotNull Condition condition, @NotNull Operation success, @NotNull Operation failure) {
         return null;
     }
 
-    // TODO: IGNITE-14389 Either implement or remove this method.
+    // TODO: IGNITE-14389 Implement.
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Entry> getAndInvoke(@NotNull Key key, @NotNull Condition condition,
-        @NotNull Operation success, @NotNull Operation failure) {
+    @Override public @NotNull CompletableFuture<Boolean> invoke(@NotNull Condition condition,
+        @NotNull Collection<Operation> success, @NotNull Collection<Operation> failure) {
         return null;
     }
 
     /** {@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))
@@ -171,7 +170,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
     }
 
     /** {@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))
@@ -180,8 +179,8 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<IgniteUuid> watch(
-        @Nullable Key keyFrom,
-        @Nullable Key keyTo,
+        @Nullable ByteArray keyFrom,
+        @Nullable ByteArray keyTo,
         long revision,
         @NotNull WatchListener lsnr
     ) {
@@ -201,7 +200,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 +209,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<IgniteUuid> watch(
-        @NotNull Collection<Key> keys,
+        @NotNull Collection<ByteArray> keys,
         long revision,
         @NotNull WatchListener lsnr
     ) {
@@ -247,7 +246,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.
@@ -311,8 +310,6 @@ public class MetaStorageServiceImpl implements MetaStorageService {
             @Override public void run() {
                 long rev = -1;
 
-                List<WatchEvent> sameRevisionEvts = new ArrayList<>();
-
                 Iterator<WatchEvent> watchEvtsIter = cursor.iterator();
 
                 while (true) {
@@ -329,19 +326,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%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/CompactedException.java
rename 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-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java
similarity index 67%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java
index 54049de..e060b8d 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Condition.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Condition.java
@@ -15,12 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
 
-import java.util.Arrays;
+import java.util.Collection;
 
 /**
- * Represents a condition for conditional update.
+ * 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. */
@@ -36,22 +39,11 @@ public final class Condition {
     }
 
     /**
-     * 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 {
+    public static final class RevisionCondition extends AbstractCondition {
         /**
          * The type of condition.
          *
@@ -62,16 +54,13 @@ public final class Condition {
         /** 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}.
+         * Constructs a condition by a revision for an entry identified by the given key.
          *
-         * @param key Key of entry, to be tested for the condition.
+         * @param key Identifies an entry which condition will be applied to.
          */
-        RevisionCondition(Key key) {
-            this.key = key;
+        RevisionCondition(byte[] key) {
+            super(key);
         }
 
         /**
@@ -176,70 +165,27 @@ public final class Condition {
             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;
-                }
-            },
+            EQUAL,
 
             /** Inequality condition type. */
-            NOT_EQUAL {
-                @Override public boolean test(long res) {
-                    return res != 0;
-                }
-            },
+            NOT_EQUAL,
 
             /** Greater than condition type. */
-            GREATER {
-                @Override public boolean test(long res) {
-                    return res > 0;
-                }
-            },
+            GREATER,
 
             /** Less than condition type. */
-            LESS {
-                @Override public boolean test(long res) {
-                    return res < 0;
-                }
-            },
+            LESS,
 
             /** Less than or equal to condition type. */
-            LESS_OR_EQUAL {
-                @Override public boolean test(long res) {
-                    return res <= 0;
-                }
-            },
+            LESS_OR_EQUAL,
 
             /** 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);
+            GREATER_OR_EQUAL
         }
     }
 
@@ -248,7 +194,7 @@ public final class Condition {
      * the one instance of condition. Subsequent invocations of any method which produces condition will throw
      * {@link IllegalStateException}.
      */
-    public static final class ValueCondition implements InnerCondition {
+    public static final class ValueCondition extends AbstractCondition {
         /**
          * The type of condition.
          *
@@ -259,16 +205,13 @@ public final class Condition {
         /** 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}.
+         * Constructs a condition by a value for an entry identified by the given key.
          *
-         * @param key Key of entry, to be tested for the condition.
+         * @param key Identifies an entry which condition will be applied to.
          */
-        ValueCondition(Key key) {
-            this.key = key;
+        ValueCondition(byte[] key) {
+            super(key);
         }
 
         /**
@@ -305,42 +248,79 @@ public final class Condition {
             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);
+        /**
+         * Defines possible condition types which can be applied to the value.
+         */
+        enum Type {
+            /** Equality condition type. */
+            EQUAL,
 
-                return type.test(res);
-            }
-            else
-               return false;
+            /** Inequality condition type. */
+            NOT_EQUAL
+        }
+    }
+
+    /**
+     * 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 {
+        /**
+         * The type of condition.
+         *
+         * @see Type
+         */
+        private Type type;
+
+        /**
+         * 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 Type#EXISTS}. This condition tests the existence of an entry
+         * identified by the given key.
+         *
+         * @return The condition of type {@link Type#EXISTS}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition exists() {
+            validate(type);
+
+            this.type = Type.EXISTS;
+
+            return new Condition(this);
+        }
+
+        /**
+         * Produces the condition of type {@link Type#NOT_EXISTS}. This condition tests the non-existence of an entry
+         * identified by the given key.
+         *
+         * @return The condition of type {@link Type#NOT_EXISTS}.
+         * @throws IllegalStateException In case when the condition is already defined.
+         */
+        public Condition notExists() {
+            validate(type);
+
+            this.type = Type.NOT_EXISTS;
+
+            return new Condition(this);
         }
 
         /**
          * 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;
-                }
-            },
+            /** Existence condition type. */
+            EXISTS,
 
-            /** 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);
+            /** Non-existence condition type. */
+            NOT_EXISTS
         }
     }
 
@@ -359,12 +339,36 @@ public final class Condition {
      */
     private interface InnerCondition {
         /**
-         * Tests the given entry on satisfaction of the condition.
+         * 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();
+    }
+
+    /**
+     * Defines an abstract condition with the key which identifies an entry which condition will be applied to.
+     */
+    private static abstract class AbstractCondition implements InnerCondition {
+        /** Entry key. */
+        private final byte[] key;
+
+        /**
+         * Constructs a condition with the given entry key.
          *
-         * @param e Entry.
-         * @return The result of condition test. {@code true} - if the entry satisfies to the condition,
-         * otherwise - {@code false}.
+         * @param key Key which identifies an entry which condition will be applied to.
          */
-        boolean test(Entry e);
+        public 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;
+        }
     }
 }
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 93%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Entry.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Entry.java
index 7f653cc..6e8832a 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.
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 2d0f9c9..09757c3 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
@@ -20,15 +20,9 @@ package org.apache.ignite.metastorage.client;
 import java.util.Collection;
 import java.util.Map;
 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,11 +36,11 @@ 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.
@@ -58,11 +52,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<Entry> get(@NotNull Key key, long revUpperBound);
+    CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound);
 
     /**
      * Retrieves entries for given keys.
@@ -71,11 +65,11 @@ public interface MetaStorageService {
      *             Collection 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(Collection<ByteArray> keys);
 
     /**
      * Retrieves entries for given keys and the revision upper bound.
@@ -88,11 +82,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<Map<Key, Entry>> getAll(Collection<Key> keys, long revUpperBound);
+    CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys, long revUpperBound);
 
     /**
      * Inserts or updates an entry with the given key and the given value.
@@ -101,11 +95,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 +109,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 +121,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 +134,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 +146,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,11 +158,11 @@ 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.
@@ -176,11 +170,11 @@ public interface MetaStorageService {
      * @param keys The keys collection. 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> removeAll(@NotNull Collection<Key> keys);
+    CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys);
 
     /**
      * Removes entries for given keys and retrieves previous entries.
@@ -190,11 +184,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 Collection<ByteArray> keys);
 
 
     /**
@@ -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,11 +292,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 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.
@@ -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 Collection<ByteArray> keys, long revision, @NotNull WatchListener lsnr);
 
     /**
      * Cancels subscription for the given identifier.
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java
similarity index 68%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java
rename to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java
index 9810fe5..bd41824 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/Operation.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/Operation.java
@@ -15,7 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.metastorage.common;
+package org.apache.ignite.metastorage.client;
+
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Defines operation for meta storage conditional update (invoke).
@@ -37,37 +39,33 @@ public final class Operation {
     /**
      * Represents operation of type <i>remove</i>.
      */
-    public static final class RemoveOp implements InnerOp {
-        /** Key. */
-        private final Key key;
-
+    public static final class RemoveOp extends AbstractOp {
         /**
-         * Creates a new remove operation for the given {@code key}.
+         * Default no-op constructor.
          *
-         * @param key Key.
+         * @param key Identifies an entry which operation will be applied to.
          */
-        RemoveOp(Key key) {
-            this.key = key;
+        RemoveOp(byte[] key) {
+            super(key);
         }
     }
 
     /**
      * Represents operation of type <i>put</i>.
      */
-    public static final class PutOp implements InnerOp {
-        /** Key. */
-        private final Key key;
-
+    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(Key key, byte[] val) {
-            this.key = key;
+        PutOp(byte[] key, byte[] val) {
+            super(key);
+
             this.val = val;
         }
     }
@@ -75,12 +73,12 @@ public final class Operation {
     /**
      * Represents operation of type <i>no-op</i>.
      */
-    public static final class NoOp implements InnerOp {
+    public static final class NoOp extends AbstractOp {
         /**
          * Default no-op constructor.
          */
         NoOp() {
-            // No-op.
+            super(null);
         }
     }
 
@@ -88,6 +86,19 @@ public final class Operation {
      * Defines operation interface.
      */
     private interface InnerOp {
-        // Marker interface.
+        @Nullable byte[] key();
+    }
+
+    private static class AbstractOp implements InnerOp {
+        @Nullable private final byte[] key;
+
+        public AbstractOp(@Nullable byte[] key) {
+            this.key = key;
+        }
+
+        @Nullable
+        @Override public byte[] key() {
+            return key;
+        }
     }
 }
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-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java
similarity index 82%
copy from modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java
copy to modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java
index 561f203..1ddac71 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatchEvent.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/metastorage/client/WatchEvent.java
@@ -15,7 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.metastorage.server;
+package org.apache.ignite.metastorage.client;
+
+import org.jetbrains.annotations.NotNull;
 
 import java.util.Collection;
 import java.util.List;
@@ -23,7 +25,7 @@ import java.util.List;
 public class WatchEvent {
     private final List<EntryEvent> entryEvts;
 
-    private final boolean batch;
+    private final boolean single;
 
     /**
      * Constructs an watch event with given entry events collection.
@@ -33,12 +35,16 @@ public class WatchEvent {
     public WatchEvent(List<EntryEvent> entryEvts) {
         assert entryEvts != null && !entryEvts.isEmpty();
 
-        this.batch = entryEvts.size() > 1;
+        this.single = entryEvts.size() == 1;
         this.entryEvts = entryEvts;
     }
 
-    public boolean batch() {
-        return batch;
+    public WatchEvent(@NotNull EntryEvent entryEvt) {
+        this(List.of(entryEvt));
+    }
+
+    public boolean single() {
+        return single;
     }
 
     public Collection<EntryEvent> entryEvents() {
@@ -46,9 +52,6 @@ public class WatchEvent {
     }
 
     public EntryEvent entryEvent() {
-        if (batch)
-            throw new IllegalStateException("Watch event represents a batch of events.");
-
         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%
rename from modules/metastorage-common/src/main/java/org/apache/ignite/metastorage/common/WatchListener.java
rename 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/DummyEntry.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java
index c4e6ed8..4dd6bad 100644
--- a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/DummyEntry.java
@@ -19,8 +19,7 @@ package org.apache.ignite.internal.metastorage.common;
 
 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.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -28,9 +27,9 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Dummy entry implementation.
  */
-public final class DummyEntry implements Entry, Serializable {
+public final class DummyEntry implements Serializable {
     /** Key. */
-    @NotNull private Key key;
+    @NotNull private ByteArray key;
 
     /** Value. */
     @Nullable private byte[] val;
@@ -48,7 +47,7 @@ public final class DummyEntry implements Entry, Serializable {
      * @param revision Revision.
      * @param updateCntr Update counter.
      */
-    public DummyEntry(@NotNull Key key, @Nullable byte[] val, long revision, long updateCntr) {
+    public DummyEntry(@NotNull ByteArray key, @Nullable byte[] val, long revision, long updateCntr) {
         this.key = key;
         this.val = val;
         this.revision = revision;
@@ -56,29 +55,30 @@ public final class DummyEntry implements Entry, Serializable {
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull Key key() {
+    public @NotNull ByteArray key() {
         return key;
     }
 
     /** {@inheritDoc} */
-    @Override public @Nullable byte[] value() {
+    public @Nullable byte[] value() {
         return val;
     }
 
     /** {@inheritDoc} */
-    @Override public long revision() {
+    public long revision() {
         return revision;
     }
 
     /** {@inheritDoc} */
-    @Override public long updateCounter() {
+    public long updateCounter() {
         return updateCntr;
     }
 
     /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
+    public boolean equals(Object o) {
         if (this == o)
             return true;
+
         if (o == null || getClass() != o.getClass())
             return false;
 
@@ -86,10 +86,13 @@ public final class DummyEntry implements Entry, Serializable {
 
         if (revision != entry.revision)
             return false;
+
         if (updateCntr != entry.updateCntr)
             return false;
+
         if (!key.equals(entry.key))
             return false;
+
         return Arrays.equals(val, entry.val);
     }
 
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..c5fcaa9 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
@@ -20,19 +20,18 @@ 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 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;
+    @NotNull private final Collection<ByteArray> keys;
 
     /** The upper bound for entry revisions. Must be positive. */
     @Nullable private Long revUpperBound;
@@ -41,7 +40,7 @@ public final class GetAllCommand implements ReadCommand {
      * @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 Collection<ByteArray> keys) {
         assert !keys.isEmpty();
 
         if (keys instanceof Serializable)
@@ -55,7 +54,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 Collection<ByteArray> keys, @NotNull Long revUpperBound) {
         this(keys);
 
         assert revUpperBound > 0;
@@ -66,7 +65,7 @@ public final class GetAllCommand implements ReadCommand {
     /**
      * @return The collection of keys.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull Collection<ByteArray> keys() {
         return keys;
     }
 
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..d586624 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
@@ -19,18 +19,17 @@ package org.apache.ignite.internal.metastorage.common.command;
 
 import java.io.Serializable;
 import java.util.List;
-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 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<ByteArray> keys;
 
     /** Values. */
     @NotNull private final List<byte[]> vals;
@@ -39,7 +38,7 @@ public final class GetAndPutAllCommand implements WriteCommand {
      * @param keys Keys.
      * @param vals Values.
      */
-    public GetAndPutAllCommand(@NotNull List<Key> keys, @NotNull List<byte[]> vals) {
+    public GetAndPutAllCommand(@NotNull List<ByteArray> keys, @NotNull List<byte[]> vals) {
         assert keys instanceof Serializable;
         assert vals instanceof Serializable;
 
@@ -50,7 +49,7 @@ public final class GetAndPutAllCommand implements WriteCommand {
     /**
      * @return Keys.
      */
-    public @NotNull List<Key> keys() {
+    public @NotNull List<ByteArray> 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..9bcb8aa 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 ByteArray key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,7 +36,7 @@ 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) {
+    public GetAndPutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
         this.key = key;
         this.val = val;
     }
@@ -45,7 +44,7 @@ public final class GetAndPutCommand implements WriteCommand {
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull ByteArray 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..5712f0b 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
@@ -20,23 +20,22 @@ 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 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 Collection<ByteArray> keys;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      */
-    public GetAndRemoveAllCommand(@NotNull Collection<Key> keys) {
+    public GetAndRemoveAllCommand(@NotNull Collection<ByteArray> keys) {
         assert !keys.isEmpty();
 
         if (keys instanceof Serializable)
@@ -48,7 +47,7 @@ public final class GetAndRemoveAllCommand implements WriteCommand {
     /**
      * @return The keys collection. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull Collection<ByteArray> 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..b4a4166 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 ByteArray key;
 
     /**
      * @param key The key. Couldn't be {@code null}.
      */
-    public GetAndRemoveCommand(@NotNull Key key) {
+    public GetAndRemoveCommand(@NotNull ByteArray key) {
         this.key = key;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull ByteArray 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..c71479b 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,27 +17,25 @@
 
 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 ByteArray 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) {
+    public GetCommand(@NotNull ByteArray key) {
         this.key = key;
     }
 
@@ -45,7 +43,7 @@ public final class GetCommand implements ReadCommand {
      * @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) {
+    public GetCommand(@NotNull ByteArray key, @NotNull long revUpperBound) {
         this.key = key;
 
         assert revUpperBound > 0;
@@ -56,14 +54,14 @@ public final class GetCommand implements ReadCommand {
     /**
      * @return Key.
      */
-    public @NotNull Key key() {
+    public @NotNull ByteArray 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/PutAllCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/PutAllCommand.java
index 0e87623..029ba99 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
@@ -20,23 +20,22 @@ package org.apache.ignite.internal.metastorage.common.command;
 import java.io.Serializable;
 import java.util.HashMap;
 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;
+    @NotNull private final Map<ByteArray, 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)
@@ -48,7 +47,7 @@ public final class PutAllCommand implements WriteCommand {
     /**
      * @return The map of keys and corresponding values. Couldn't be  or empty.
      */
-    public @NotNull Map<Key, byte[]> values() {
+    public @NotNull Map<ByteArray, 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..1efff24 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 ByteArray key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,7 +36,7 @@ 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) {
+    public PutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
         this.key = key;
         this.val = val;
     }
@@ -45,7 +44,7 @@ public final class PutCommand implements WriteCommand {
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull ByteArray 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..8053f69 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 ByteArray keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final Key keyTo;
+    @Nullable private final ByteArray 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,9 +50,9 @@ 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,
+        @NotNull long revUpperBound
     ) {
         this.keyFrom = keyFrom;
         this.keyTo = keyTo;
@@ -63,21 +62,21 @@ public final class RangeCommand implements WriteCommand {
     /**
      * @return Start key of range (inclusive). Couldn't be {@code null}.
      */
-    public @NotNull Key keyFrom() {
+    public @NotNull ByteArray keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable Key keyTo() {
+    public @Nullable ByteArray 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..4edad35 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
@@ -20,22 +20,21 @@ 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 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;
+    @NotNull private final Collection<ByteArray> keys;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      */
-    public RemoveAllCommand(@NotNull Collection<Key> keys) {
+    public RemoveAllCommand(@NotNull Collection<ByteArray> keys) {
         if (keys instanceof Serializable)
             this.keys = keys;
         else
@@ -45,7 +44,7 @@ public final class RemoveAllCommand implements WriteCommand {
     /**
      * @return The keys collection. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull Collection<ByteArray> 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..18322b9 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 ByteArray key;
 
     /**
      * @param key he key. Couldn't be {@code null}.
      */
-    public RemoveCommand(@NotNull Key key) {
+    public RemoveCommand(@NotNull ByteArray key) {
         this.key = key;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull Key key() {
+    public @NotNull ByteArray key() {
         return key;
     }
 }
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..9a0c628 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
@@ -20,17 +20,16 @@ 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 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;
+    @NotNull private final Collection<ByteArray> keys;
 
     /** Start revision inclusive. {@code 0} - all revisions. */
     @NotNull private final Long revision;
@@ -39,7 +38,7 @@ public final class WatchExactKeysCommand implements WriteCommand {
      * @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) {
+    public WatchExactKeysCommand(@NotNull Collection<ByteArray> keys, @NotNull Long revision) {
         if (keys instanceof Serializable)
             this.keys = keys;
         else
@@ -51,7 +50,7 @@ public final class WatchExactKeysCommand implements WriteCommand {
     /**
      * @return The keys collection. Couldn't be {@code null}.
      */
-    public @NotNull Collection<Key> keys() {
+    public @NotNull Collection<ByteArray> 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..f164642 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,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;
 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 ByteArray keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final Key keyTo;
+    @Nullable private final ByteArray 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,9 +49,9 @@ 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,
+        @NotNull long revision
     ) {
         this.keyFrom = keyFrom;
         this.keyTo = keyTo;
@@ -62,21 +61,21 @@ public final class WatchRangeKeysCommand implements WriteCommand {
     /**
      * @return Start key of range (inclusive). Couldn't be {@code null}.
      */
-    public @Nullable Key keyFrom() {
+    public @Nullable ByteArray keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable Key keyTo() {
+    public @Nullable ByteArray 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/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-server/pom.xml b/modules/metastorage-server/pom.xml
index d73d080..5f0f453 100644
--- a/modules/metastorage-server/pom.xml
+++ b/modules/metastorage-server/pom.xml
@@ -29,20 +29,18 @@
         <relativePath>../../parent/pom.xml</relativePath>
     </parent>
 
-    <artifactId>metastorage-server</artifactId>
+    <artifactId>ignite-metastorage-server</artifactId>
     <version>3.0.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-core</artifactId>
-            <version>${project.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>metastorage-common</artifactId>
-            <version>${project.version}</version>
+            <artifactId>ignite-metastorage-common</artifactId>
         </dependency>
 
         <dependency>
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-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
similarity index 56%
copy from modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
copy to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
index 554a3a7..e20459a 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Condition.java
@@ -17,42 +17,28 @@
 
 package org.apache.ignite.internal.metastorage.server;
 
-/**
- * Represent an change event for particular key and entry.
- */
-public class EntryEvent {
-    /** Old (previous) entry. */
-    private final Entry oldEntry;
+import org.jetbrains.annotations.NotNull;
 
-    /** New (current) entry. */
-    private final Entry entry;
-
-    /**
-     * Constructs event with given old and new entries.
-     *
-     * @param oldEntry Old entry.
-     * @param curEntry New entry.
-     */
-    EntryEvent(Entry oldEntry, Entry curEntry) {
-        this.oldEntry = oldEntry;
-        this.entry = curEntry;
-    }
+import java.util.Collection;
 
+/**
+ * Defines interface for condition which could be applied to an entry.
+ * 
+ * @see KeyValueStorage#invoke(Condition, Collection, Collection) 
+ */
+public interface Condition {
     /**
-     * Returns old entry.
+     * Returns the key which identifies an entry which condition will applied to.
      *
-     * @return Old entry.
+     * @return The key which identifies an entry which condition will applied to.
      */
-    public Entry oldEntry() {
-        return oldEntry;
-    }
+    @NotNull byte[] key();
 
     /**
-     * Rreturns new entry.
+     * Tests the given entry on condition.
      *
-     * @return New entry.
+     * @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}.
      */
-    public Entry entry() {
-        return entry;
-    }
+    boolean test(@NotNull Entry e);
 }
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
index 554a3a7..c1e4f56 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.metastorage.server;
 
 /**
- * Represent an change event for particular key and entry.
+ * Represent an update event for particular key and entry.
  */
 public class EntryEvent {
     /** Old (previous) 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
index 5d6da44..f46610c 100644
--- 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
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.metastorage.server;
 
 import java.util.Collection;
 import java.util.List;
-import org.apache.ignite.metastorage.common.Cursor;
+import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.NotNull;
 
 public interface KeyValueStorage {
@@ -59,6 +59,8 @@ public interface KeyValueStorage {
     @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);
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..aaea75b
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Operation.java
@@ -0,0 +1,111 @@
+/*
+ * 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.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 Type#NO_OP}.</li>
+ *     <li>Only {@link Type#PUT} operation contains value which will be written to meta storage.</li>
+ * </ul>
+ */
+final class Operation {
+    /**
+     * Key identifies an entry which operation will be applied to. Key is {@code null} for {@link Type#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 Type#PUT}
+     * operation.
+     */
+    @Nullable
+    private final byte[] val;
+
+    /**
+     * Operation type.
+     * @see Type
+     */
+    @NotNull
+    private final Type 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}.
+     */
+    Operation(@NotNull Type type, @Nullable byte[] key, @Nullable byte[] val) {
+        assert (type == Type.NO_OP && key == null && val == null)
+                || (type == Type.PUT && key != null && val != null)
+                || (type == Type.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 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 byte[] value() {
+        return val;
+    }
+
+    /**
+     * Returns an operation type.
+     *
+     * @return An operation type.
+     */
+    @NotNull Type type() {
+        return type;
+    }
+
+    /** Defines operation types. */
+    enum Type {
+        /** Put operation. */
+        PUT,
+
+        /** Remove operation. */
+        REMOVE,
+
+        /** No-op operation. */
+        NO_OP
+    }
+}
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
index b37c96a..1f3f0e4 100644
--- 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
@@ -29,7 +29,7 @@ import java.util.NoSuchElementException;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.function.Predicate;
-import org.apache.ignite.metastorage.common.Cursor;
+import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.TestOnly;
 
@@ -63,14 +63,22 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
     @Override public void put(byte[] key, byte[] value) {
         synchronized (mux) {
-            doPut(key, value);
+            long curRev = rev + 1;
+
+            doPut(key, value, curRev);
+
+            rev = curRev;
         }
     }
 
     @NotNull
     @Override public Entry getAndPut(byte[] key, byte[] bytes) {
         synchronized (mux) {
-            long lastRev = doPut(key, bytes);
+            long curRev = rev + 1;
+
+            long lastRev = doPut(key, bytes, curRev);
+
+            rev = curRev;
 
             // Return previous value.
             return doGetValue(key, lastRev);
@@ -129,15 +137,24 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     @Override
     public void remove(byte[] key) {
         synchronized (mux) {
-            Entry e = doGet(key, LATEST_REV, false);
-
-            if (e.empty() || e.tombstone())
-                return;
+            long curRev = rev + 1;
 
-            doPut(key, TOMBSTONE);
+            if (doRemove(key, curRev))
+                rev = curRev;
         }
     }
 
+    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;
+    }
+
     @NotNull
     @Override public Entry getAndRemove(byte[] key) {
         synchronized (mux) {
@@ -204,6 +221,47 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return res;
     }
 
+    @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;
+        }
+    }
+
     @Override public Cursor<Entry> range(byte[] keyFrom, byte[] keyTo) {
         return new RangeCursor(keyFrom, keyTo, rev);
     }
@@ -365,9 +423,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return new Entry(key, lastVal.bytes() , lastRev, lastVal.updateCounter());
     }
 
-    private long doPut(byte[] key, byte[] bytes) {
-        long curRev = ++rev;
-
+    private long doPut(byte[] key, byte[] bytes, long curRev) {
         long curUpdCntr = ++updCntr;
 
         // Update keysIdx.
@@ -378,13 +434,25 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         revs.add(curRev);
 
         // Update revsIdx.
-        NavigableMap<byte[], Value> entries = new TreeMap<>(CMP);
+        //NavigableMap<byte[], Value> entries = new TreeMap<>(CMP);
 
         Value val = new Value(bytes, curUpdCntr);
 
-        entries.put(key, val);
+        //entries.put(key, val);
+
+        //revsIdx.put(curRev, entries);
 
-        revsIdx.put(curRev, entries);
+        revsIdx.compute(
+                curRev,
+                (rev, entries) -> {
+                    if (entries == null)
+                        entries = new TreeMap<>(CMP);
+
+                    entries.put(key, val);
+
+                    return entries;
+                }
+        );
 
         return lastRev;
     }
@@ -437,6 +505,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         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;
@@ -445,14 +514,32 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
             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<>() {
                 @Override public boolean hasNext() {
                     synchronized (mux) {
@@ -530,20 +617,39 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
     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
+            // TODO: implement.
         }
 
+        /** {@inheritDoc} */
         @NotNull
         @Override public Iterator<WatchEvent> iterator() {
+            return it;
+        }
+
+        @NotNull
+        Iterator<WatchEvent> createIterator() {
             return new Iterator<>() {
                 @Override public boolean hasNext() {
                     synchronized (mux) {
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..444bfdb
--- /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.
+     */
+    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
index 561f203..4f2f305 100644
--- 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
@@ -23,7 +23,7 @@ import java.util.List;
 public class WatchEvent {
     private final List<EntryEvent> entryEvts;
 
-    private final boolean batch;
+    private final boolean single;
 
     /**
      * Constructs an watch event with given entry events collection.
@@ -33,12 +33,12 @@ public class WatchEvent {
     public WatchEvent(List<EntryEvent> entryEvts) {
         assert entryEvts != null && !entryEvts.isEmpty();
 
-        this.batch = entryEvts.size() > 1;
+        this.single = entryEvts.size() == 1;
         this.entryEvts = entryEvts;
     }
 
-    public boolean batch() {
-        return batch;
+    public boolean single() {
+        return single;
     }
 
     public Collection<EntryEvent> entryEvents() {
@@ -46,9 +46,6 @@ public class WatchEvent {
     }
 
     public EntryEvent entryEvent() {
-        if (batch)
-            throw new IllegalStateException("Watch event represents a batch of events.");
-
         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 89%
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..71b4f34 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,7 +15,7 @@
  * 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;
@@ -41,16 +41,14 @@ import org.apache.ignite.internal.metastorage.common.command.WatchRangeKeysComma
 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.Entry;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.WatchEvent;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.ByteArray;
 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;
@@ -68,7 +66,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<@NotNull Cursor<?>, @NotNull Iterator<?>>> cursors;
 
     /**
      * @param storage Storage.
@@ -87,7 +85,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 if (clo.command() instanceof GetCommand) {
                     GetCommand getCmd = (GetCommand)clo.command();
 
-                    if (getCmd.revision() != null)
+                    if (getCmd.revision() != 0)
                         clo.success(storage.get(getCmd.key().bytes(), getCmd.revision()));
                     else
                         clo.success(storage.get(getCmd.key().bytes()));
@@ -97,13 +95,13 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
 
                     if (getAllCmd.revision() != null) {
                         clo.success(storage.getAll(
-                            getAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()),
+                            getAllCmd.keys().stream().map(ByteArray::bytes).collect(Collectors.toList()),
                             getAllCmd.revision())
                         );
                     }
                     else {
                         clo.success(storage.getAll(
-                            getAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()))
+                            getAllCmd.keys().stream().map(ByteArray::bytes).collect(Collectors.toList()))
                         );
                     }
                 }
@@ -117,7 +115,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 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);
 
@@ -153,7 +151,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                     PutAllCommand putAllCmd = (PutAllCommand)clo.command();
 
                     storage.putAll(
-                        putAllCmd.values().keySet().stream().map(Key::bytes).collect(Collectors.toList()),
+                        putAllCmd.values().keySet().stream().map(ByteArray::bytes).collect(Collectors.toList()),
                         new ArrayList<>(putAllCmd.values().values()));
 
                     clo.success(null);
@@ -162,7 +160,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                     GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand)clo.command();
 
                     Collection<Entry> entries = storage.getAndPutAll(
-                        getAndPutAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList()),
+                        getAndPutAllCmd.keys().stream().map(ByteArray::bytes).collect(Collectors.toList()),
                         getAndPutAllCmd.vals()
                     );
 
@@ -186,7 +184,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 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().stream().map(ByteArray::bytes).collect(Collectors.toList()));
 
                     clo.success(null);
                 }
@@ -194,7 +192,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                     GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand)clo.command();
 
                     Collection<Entry> entries = storage.getAndRemoveAll(
-                        getAndRmvAllCmd.keys().stream().map(Key::bytes).collect(Collectors.toList())
+                        getAndRmvAllCmd.keys().stream().map(ByteArray::bytes).collect(Collectors.toList())
                     );
 
                     if (!(entries instanceof Serializable))
@@ -268,7 +266,7 @@ 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.keys().stream().map(ByteArray::bytes).collect(Collectors.toList()),
                         watchCmd.revision());
 
                     cursors.put(
@@ -281,7 +279,7 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 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);
 
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..7d58743
--- /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
index 27df790..8e138fd 100644
--- 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
@@ -23,13 +23,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.stream.Collectors;
-import org.apache.ignite.metastorage.common.Cursor;
-import org.apache.ignite.metastorage.common.Key;
+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.*;
+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;
 
 class SimpleInMemoryKeyValueStorageTest {
     private KeyValueStorage storage;
@@ -108,10 +114,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(1, e1.revision());
@@ -121,7 +127,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val1, e1.value());
 
         // Test rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(3, e2.revision());
@@ -131,7 +137,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_2, e2.value());
 
         // Test removed value.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(5, e3.revision());
@@ -140,7 +146,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -183,10 +189,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(1, e1.revision());
@@ -196,7 +202,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val1, e1.value());
 
         // Test while not rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(2, e2.revision());
@@ -206,12 +212,12 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_1, e2.value());
 
         // Values with larger revision don't exist yet.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertTrue(e3.empty());
 
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertTrue(e4.empty());
@@ -221,10 +227,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        e1 = map.get(new Key(key1));
+        e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(1, e1.revision());
@@ -234,7 +240,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val1, e1.value());
 
         // Test rewritten value.
-        e2 = map.get(new Key(key2));
+        e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(3, e2.revision());
@@ -244,7 +250,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_2, e2.value());
 
         // Test not removed value.
-        e3 = map.get(new Key(key3));
+        e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(4, e3.revision());
@@ -254,7 +260,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val3, e3.value());
 
         // Value with larger revision doesn't exist yet.
-        e4 = map.get(new Key(key4));
+        e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertTrue(e4.empty());
@@ -325,10 +331,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(4, e1.revision());
@@ -338,7 +344,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val1, e1.value());
 
         // Test rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(4, e2.revision());
@@ -348,7 +354,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_2, e2.value());
 
         // Test removed value.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(4, e3.revision());
@@ -357,7 +363,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -399,10 +405,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(3, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(0, e1.revision());
@@ -411,7 +417,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertTrue(e1.empty());
 
         // Test rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(1, e2.revision());
@@ -421,7 +427,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_1, e2.value());
 
         // Test removed value.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(3, e3.revision());
@@ -434,10 +440,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        e1 = map.get(new Key(key1));
+        e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(4, e1.revision());
@@ -447,7 +453,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val1, e1.value());
 
         // Test rewritten value.
-        e2 = map.get(new Key(key2));
+        e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(4, e2.revision());
@@ -457,7 +463,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_2, e2.value());
 
         // Test removed value.
-        e3 = map.get(new Key(key3));
+        e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(4, e3.revision());
@@ -466,7 +472,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -618,10 +624,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(6, e1.revision());
@@ -630,7 +636,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e1.empty());
 
         // Test rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(6, e2.revision());
@@ -639,7 +645,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e2.empty());
 
         // Test removed value.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(5, e3.revision());
@@ -648,7 +654,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -693,10 +699,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        Map<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        Map<ByteArray, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        Entry e1 = map.get(new Key(key1));
+        Entry e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(1, e1.revision());
@@ -706,7 +712,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
 
         // Test rewritten value.
-        Entry e2 = map.get(new Key(key2));
+        Entry e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(3, e2.revision());
@@ -716,7 +722,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
 
         // Test removed value.
-        Entry e3 = map.get(new Key(key3));
+        Entry e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(5, e3.revision());
@@ -725,7 +731,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        Entry e4 = map.get(new Key(key4));
+        Entry e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -736,10 +742,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         assertEquals(4, entries.size());
 
-        map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), identity()));
+        map =  entries.stream().collect(Collectors.toMap(e -> new ByteArray(e.key()), identity()));
 
         // Test regular put value.
-        e1 = map.get(new Key(key1));
+        e1 = map.get(new ByteArray(key1));
 
         assertNotNull(e1);
         assertEquals(6, e1.revision());
@@ -748,7 +754,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e1.empty());
 
         // Test rewritten value.
-        e2 = map.get(new Key(key2));
+        e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
         assertEquals(6, e2.revision());
@@ -757,7 +763,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e2.empty());
 
         // Test removed value.
-        e3 = map.get(new Key(key3));
+        e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
         assertEquals(5, e3.revision());
@@ -766,7 +772,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertFalse(e3.empty());
 
         // Test empty value.
-        e4 = map.get(new Key(key4));
+        e4 = map.get(new ByteArray(key4));
 
         assertNotNull(e4);
         assertFalse(e4.tombstone());
@@ -939,6 +945,553 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @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(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.PUT, key2, val2)
+                ),
+                List.of(new Operation(Operation.Type.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(Operation.Type.PUT, key3, val3)),
+                List.of(
+                        new Operation(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.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(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.PUT, key2, val2)
+                ),
+                List.of(new Operation(Operation.Type.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(Operation.Type.PUT, key3, val3)),
+                List.of(
+                        new Operation(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.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(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.PUT, key2, val2)
+                ),
+                List.of(new Operation(Operation.Type.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(Operation.Type.PUT, key3, val3)),
+                List.of(
+                        new Operation(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.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(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.PUT, key2, val2)
+                ),
+                List.of(new Operation(Operation.Type.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(Operation.Type.PUT, key3, val3)),
+                List.of(
+                        new Operation(Operation.Type.PUT, key1, val1_2),
+                        new Operation(Operation.Type.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(Operation.Type.NO_OP, null, null)),
+                List.of(new Operation(Operation.Type.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(Operation.Type.PUT, key2, val2),
+                        new Operation(Operation.Type.PUT, key3, val3)
+                ),
+                List.of(new Operation(Operation.Type.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(Operation.Type.REMOVE, key2, null),
+                        new Operation(Operation.Type.REMOVE, key3, null)
+                ),
+                List.of(new Operation(Operation.Type.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());
@@ -1155,15 +1708,15 @@ class SimpleInMemoryKeyValueStorageTest {
 
         WatchEvent watchEvent = it.next();
 
-        assertTrue(watchEvent.batch());
+        assertFalse(watchEvent.single());
 
-        Map<Key, EntryEvent> map = watchEvent.entryEvents().stream()
-                .collect(Collectors.toMap(evt -> new Key(evt.entry().key()), identity()));
+        Map<ByteArray, EntryEvent> map = watchEvent.entryEvents().stream()
+                .collect(Collectors.toMap(evt -> new ByteArray(evt.entry().key()), identity()));
 
         assertEquals(2, map.size());
 
         // First update under revision.
-        EntryEvent e2 = map.get(new Key(key2));
+        EntryEvent e2 = map.get(new ByteArray(key2));
 
         assertNotNull(e2);
 
@@ -1186,7 +1739,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(val2_2, newEntry2.value());
 
         // Second update under revision.
-        EntryEvent e3 = map.get(new Key(key3));
+        EntryEvent e3 = map.get(new ByteArray(key3));
 
         assertNotNull(e3);
 
@@ -1213,7 +1766,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         watchEvent = it.next();
 
-        assertFalse(watchEvent.batch());
+        assertTrue(watchEvent.single());
 
         EntryEvent e1 = watchEvent.entryEvent();
 
@@ -1269,7 +1822,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         WatchEvent watchEvent = it.next();
 
-        assertFalse(watchEvent.batch());
+        assertTrue(watchEvent.single());
 
         EntryEvent e1 = watchEvent.entryEvent();
 
@@ -1299,7 +1852,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         watchEvent = it.next();
 
-        assertFalse(watchEvent.batch());
+        assertTrue(watchEvent.single());
 
         e1 = watchEvent.entryEvent();
 
@@ -1328,7 +1881,6 @@ class SimpleInMemoryKeyValueStorageTest {
     public void watchCursorForKeys() {
         byte[] key1 = k(1);
         byte[] val1_1 = kv(1, 11);
-        byte[] val1_2 = kv(1, 12);
 
         byte[] key2 = k(2);
         byte[] val2_1 = kv(2, 21);
@@ -1358,7 +1910,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         WatchEvent watchEvent = it.next();
 
-        assertTrue(watchEvent.batch());
+        assertFalse(watchEvent.single());
 
         assertFalse(it.hasNext());
 
@@ -1368,7 +1920,7 @@ class SimpleInMemoryKeyValueStorageTest {
 
         watchEvent = it.next();
 
-        assertFalse(watchEvent.batch());
+        assertTrue(watchEvent.single());
 
         assertFalse(it.hasNext());
 
diff --git a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ValueConditionTest.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ValueConditionTest.java
new file mode 100644
index 0000000..654c67e
--- /dev/null
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/ValueConditionTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.ValueCondition.Type.EQUAL;
+import static org.apache.ignite.internal.metastorage.server.ValueCondition.Type.NOT_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for entry value condition.
+ *
+ * @see ValueCondition
+ */
+public class ValueConditionTest {
+    /** Entry key. */
+    private static final byte[] KEY = new byte[] {1};
+
+    /** Entry value. */
+    private static final byte[] VAL_1 = new byte[] {11};
+
+    /** Other entry value. */
+    private static final byte[] VAL_2 = new byte[] {22};
+
+    /**
+     * Tests values equality.
+     */
+    @Test
+    public void eq() {
+        Condition cond = new ValueCondition(EQUAL, KEY, VAL_1);
+
+        assertTrue(cond.test(new Entry(KEY, VAL_1, 1, 1)));
+    }
+
+    /**
+     * Tests values inequality.
+     */
+    @Test
+    public void ne() {
+        Condition cond = new ValueCondition(NOT_EQUAL, KEY, VAL_1);
+
+        assertTrue(cond.test(new Entry(KEY, VAL_2, 1, 1)));
+    }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index d6e15e8..f631db8 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.metastorage;
 
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
@@ -32,6 +31,7 @@ import org.apache.ignite.configuration.schemas.runner.NodeConfiguration;
 import org.apache.ignite.internal.metastorage.client.MetaStorageServiceImpl;
 import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
 import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -39,16 +39,12 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
 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.KeyValueStorageImpl;
-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.metastorage.common.raft.MetaStorageCommandListener;
+import org.apache.ignite.metastorage.client.CompactedException;
+import org.apache.ignite.metastorage.client.Condition;
+import org.apache.ignite.metastorage.client.Entry;
+import org.apache.ignite.metastorage.client.Operation;
+import org.apache.ignite.metastorage.client.OperationTimeoutException;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterService;
 import org.jetbrains.annotations.NotNull;
@@ -131,6 +127,9 @@ public class MetaStorageManager {
             clusterNode -> Arrays.asList(metastorageNodes).contains(clusterNode.name());
 
         if (hasMetastorage(locNodeName, metastorageNodes)) {
+
+            //TODO:
+/*
             this.metaStorageSvcFut = CompletableFuture.completedFuture(new MetaStorageServiceImpl(
                     raftMgr.startRaftGroup(
                         METASTORAGE_RAFT_GROUP_NAME,
@@ -141,6 +140,7 @@ public class MetaStorageManager {
                     )
                 )
             );
+*/
         }
         else if (metastorageNodes.length > 0) {
             this.metaStorageSvcFut = CompletableFuture.completedFuture(new MetaStorageServiceImpl(
@@ -202,7 +202,7 @@ public class MetaStorageManager {
      * subscription
      */
     public synchronized CompletableFuture<Long> registerWatch(
-        @Nullable Key key,
+        @Nullable ByteArray key,
         @NotNull WatchListener lsnr
     ) {
         return waitForReDeploy(watchAggregator.add(key, lsnr));
@@ -217,7 +217,7 @@ public class MetaStorageManager {
      * subscription
      */
     public synchronized CompletableFuture<Long> registerWatchByPrefix(
-        @Nullable Key key,
+        @Nullable ByteArray key,
         @NotNull WatchListener lsnr
     ) {
         return waitForReDeploy(watchAggregator.addPrefix(key, lsnr));
@@ -232,7 +232,7 @@ public class MetaStorageManager {
      * subscription
      */
     public synchronized CompletableFuture<Long> registerWatch(
-        @NotNull Collection<Key> keys,
+        @NotNull Collection<ByteArray> keys,
         @NotNull WatchListener lsnr
     ) {
         return waitForReDeploy(watchAggregator.add(keys, lsnr));
@@ -247,8 +247,8 @@ public class MetaStorageManager {
      * @return future with id of registered watch.
      */
     public synchronized CompletableFuture<Long> registerWatch(
-        @NotNull Key from,
-        @NotNull Key to,
+        @NotNull ByteArray from,
+        @NotNull ByteArray to,
         @NotNull WatchListener lsnr
     ) {
         return waitForReDeploy(watchAggregator.add(from, to, lsnr));
@@ -269,132 +269,120 @@ public class MetaStorageManager {
     }
 
     /**
-     * @see MetaStorageService#get(Key)
+     * @see MetaStorageService#get(ByteArray)
      */
-    public @NotNull CompletableFuture<Entry> get(@NotNull Key key) {
+    public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key) {
         return metaStorageSvcFut.thenCompose(svc -> svc.get(key));
     }
 
     /**
-     * @see MetaStorageService#get(Key, long)
+     * @see MetaStorageService#get(ByteArray, long)
      */
-    public @NotNull CompletableFuture<Entry> get(@NotNull Key key, long revUpperBound) {
+    public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound) {
         return metaStorageSvcFut.thenCompose(svc -> svc.get(key, revUpperBound));
     }
 
     /**
      * @see MetaStorageService#getAll(Collection)
      */
-    public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys) {
+    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAll(keys));
     }
 
     /**
      * @see MetaStorageService#getAll(Collection, long)
      */
-    public @NotNull CompletableFuture<Map<Key, Entry>> getAll(Collection<Key> keys, long revUpperBound) {
+    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys, long revUpperBound) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAll(keys, revUpperBound));
     }
 
     /**
-     * @see MetaStorageService#put(Key, byte[])
+     * @see MetaStorageService#put(ByteArray, byte[])
      */
-    public @NotNull CompletableFuture<Void> put(@NotNull Key key, byte[] val) {
+    public @NotNull CompletableFuture<Void> put(@NotNull ByteArray key, byte[] val) {
         return metaStorageSvcFut.thenCompose(svc -> svc.put(key, val));
     }
 
     /**
-     * @see MetaStorageService#getAndPut(Key, byte[])
+     * @see MetaStorageService#getAndPut(ByteArray, byte[])
      */
-    public @NotNull CompletableFuture<Entry> getAndPut(@NotNull Key key, byte[] val) {
+    public @NotNull CompletableFuture<Entry> getAndPut(@NotNull ByteArray key, byte[] val) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAndPut(key, val));
     }
 
     /**
      * @see MetaStorageService#putAll(Map)
      */
-    public @NotNull CompletableFuture<Void> putAll(@NotNull Map<Key, byte[]> vals) {
+    public @NotNull CompletableFuture<Void> putAll(@NotNull Map<ByteArray, byte[]> vals) {
         return metaStorageSvcFut.thenCompose(svc -> svc.putAll(vals));
     }
 
     /**
      * @see MetaStorageService#getAndPutAll(Map)
      */
-    public @NotNull CompletableFuture<Map<Key, Entry>> getAndPutAll(@NotNull Map<Key, byte[]> vals) {
+    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndPutAll(@NotNull Map<ByteArray, byte[]> vals) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAndPutAll(vals));
     }
 
     /**
-     * @see MetaStorageService#remove(Key)
+     * @see MetaStorageService#remove(ByteArray)
      */
-    public @NotNull CompletableFuture<Void> remove(@NotNull Key key) {
+    public @NotNull CompletableFuture<Void> remove(@NotNull ByteArray key) {
         return metaStorageSvcFut.thenCompose(svc -> svc.remove(key));
     }
 
     /**
-     * @see MetaStorageService#getAndRemove(Key)
+     * @see MetaStorageService#getAndRemove(ByteArray)
      */
-    public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull Key key) {
+    public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull ByteArray key) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAndRemove(key));
     }
 
     /**
      * @see MetaStorageService#removeAll(Collection)
      */
-    public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<Key> keys) {
+    public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys) {
         return metaStorageSvcFut.thenCompose(svc -> svc.removeAll(keys));
     }
 
     /**
      * @see MetaStorageService#getAndRemoveAll(Collection)
      */
-    public @NotNull CompletableFuture<Map<Key, Entry>> getAndRemoveAll(@NotNull Collection<Key> keys) {
+    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Collection<ByteArray> keys) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAndRemoveAll(keys));
     }
 
     /**
      * Invoke with single success/failure operation.
      *
-     * @see MetaStorageService#invoke(Condition, Collection, Collection)
+     * @see MetaStorageService#invoke(Condition, Operation, Operation)
      */
     public @NotNull CompletableFuture<Boolean> invoke(
         @NotNull Condition cond,
         @NotNull Operation success,
         @NotNull Operation failure
     ) {
-        return metaStorageSvcFut.thenCompose(svc -> svc.invoke(cond, Collections.singletonList(success), Collections.singletonList(failure)));
+        return metaStorageSvcFut.thenCompose(svc -> svc.invoke(cond, success, failure));
     }
 
     /**
      * @see MetaStorageService#invoke(Condition, Collection, Collection)
      */
     public @NotNull CompletableFuture<Boolean> invoke(
-        @NotNull Condition cond,
-        @NotNull Collection<Operation> success,
-        @NotNull Collection<Operation> failure
+            @NotNull Condition cond,
+            @NotNull Collection<Operation> success,
+            @NotNull Collection<Operation> failure
     ) {
         return metaStorageSvcFut.thenCompose(svc -> svc.invoke(cond, success, failure));
     }
 
     /**
-     * @see MetaStorageService#getAndInvoke(Key, Condition, Operation, Operation)
+     * @see MetaStorageService#range(ByteArray, ByteArray, long)
      */
-    public @NotNull CompletableFuture<Entry> getAndInvoke(
-        @NotNull Key key,
-        @NotNull Condition cond,
-        @NotNull Operation success,
-        @NotNull Operation failure
-    ) {
-        return metaStorageSvcFut.thenCompose(svc -> svc.getAndInvoke(key, cond, success, failure));
-    }
-
-    /**
-     * @see MetaStorageService#range(Key, Key, long)
-     */
-    public @NotNull Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo, long revUpperBound) {
+    public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound) {
         return new CursorWrapper<>(
-            metaStorageSvcFut,
-            metaStorageSvcFut.thenApply(svc -> svc.range(keyFrom, keyTo, revUpperBound))
+                metaStorageSvcFut,
+                metaStorageSvcFut.thenApply(svc -> svc.range(keyFrom, keyTo, revUpperBound))
         );
     }
 
@@ -408,10 +396,10 @@ public class MetaStorageManager {
      * @return Cursor built upon entries corresponding to the given range and applied revision.
      * @throws OperationTimeoutException If the operation is timed out.
      * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
-     * @see Key
+     * @see ByteArray
      * @see Entry
      */
-    public @NotNull Cursor<Entry> rangeWithAppliedRevision(@NotNull Key keyFrom, @Nullable Key keyTo) {
+    public @NotNull Cursor<Entry> rangeWithAppliedRevision(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo) {
         return new CursorWrapper<>(
             metaStorageSvcFut,
             metaStorageSvcFut.thenApply(svc -> {
@@ -426,9 +414,9 @@ public class MetaStorageManager {
     }
 
     /**
-     * @see MetaStorageService#range(Key, Key)
+     * @see MetaStorageService#range(ByteArray, ByteArray)
      */
-    public @NotNull Cursor<Entry> range(@NotNull Key keyFrom, @Nullable Key keyTo) {
+    public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo) {
         return new CursorWrapper<>(
             metaStorageSvcFut,
             metaStorageSvcFut.thenApply(svc -> svc.range(keyFrom, keyTo))
@@ -484,13 +472,14 @@ public class MetaStorageManager {
      * @param revision associated revision.
      * @return future, which will be completed when store action finished.
      */
-    private CompletableFuture<Void> storeEntries(Collection<IgniteBiTuple<Key, byte[]>> entries, long revision) {
+    private CompletableFuture<Void> storeEntries(Collection<IgniteBiTuple<ByteArray, byte[]>> entries, long revision) {
         try {
             return vaultMgr.putAll(entries.stream().collect(
                 Collectors.toMap(
-                    e -> ByteArray.fromString(e.getKey().toString()),
-                    IgniteBiTuple::getValue)),
-                revision);
+                        e -> ByteArray.fromString(e.getKey().toString()),
+                        IgniteBiTuple::getValue)
+                    ),
+                    revision);
         }
         catch (IgniteInternalCheckedException e) {
             throw new IgniteInternalException("Couldn't put entries with considered revision.", e);
@@ -563,6 +552,8 @@ public class MetaStorageManager {
         /** Inner iterator future. */
         private final CompletableFuture<Iterator<T>> innerIterFut;
 
+        private final InnerIterator it = new InnerIterator();
+
         /**
          * @param metaStorageSvcFut Meta storage service future.
          * @param innerCursorFut Inner cursor future.
@@ -592,27 +583,38 @@ public class MetaStorageManager {
 
         /** {@inheritDoc} */
         @NotNull @Override public Iterator<T> iterator() {
-            return new Iterator<>() {
-                /** {@inheritDoc} */
-                @Override public boolean hasNext() {
-                    try {
-                        return innerIterFut.thenApply(Iterator::hasNext).get();
-                    }
-                    catch (InterruptedException | ExecutionException e) {
-                        throw new IgniteInternalException(e);
-                    }
+            return it;
+        }
+
+        @Override
+        public boolean hasNext() {
+            return it.hasNext();
+        }
+
+        @Override
+        public T next() {
+            return it.next();
+        }
+
+        private class InnerIterator implements Iterator<T> {
+            @Override public boolean hasNext() {
+                try {
+                    return innerIterFut.thenApply(Iterator::hasNext).get();
                 }
+                catch (InterruptedException | ExecutionException e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
 
-                /** {@inheritDoc} */
-                @Override public T next() {
-                    try {
-                        return innerIterFut.thenApply(Iterator::next).get();
-                    }
-                    catch (InterruptedException | ExecutionException e) {
-                        throw new IgniteInternalException(e);
-                    }
+            /** {@inheritDoc} */
+            @Override public T next() {
+                try {
+                    return innerIterFut.thenApply(Iterator::next).get();
                 }
-            };
+                catch (InterruptedException | ExecutionException e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
         }
     }
 }
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
index 754bf76..cfedfbf 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.metastorage.watch;
 
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.metastorage.client.WatchListener;
 
 /**
  * Watch implementation with associated revision.
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
index 654b872..622589b 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.metastorage.watch;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.metastorage.common.Key;
 
 /**
  * Filter for listen key's changes on metastore.
@@ -32,38 +32,38 @@ public interface KeyCriterion {
      *
      * @return Ignite tuple with first key as start of range and second as the end.
      */
-    public IgniteBiTuple<Key, Key> toRange();
+    public IgniteBiTuple<ByteArray, ByteArray> toRange();
 
     /**
      * Check if this key criterion contains the key.
      *
      * @return true if criterion contains the key, false otherwise.
      */
-    public boolean contains(Key key);
+    public boolean contains(ByteArray key);
 
     /**
      * Simple criterion which contains exactly one key.
      */
     static class ExactCriterion implements KeyCriterion {
         /** The key of criterion. */
-        private final Key key;
+        private final ByteArray key;
 
         /**
          * Creates the instance of exact criterion.
          *
          * @param key Instance of the reference key.
          */
-        public ExactCriterion(Key key) {
+        public ExactCriterion(ByteArray key) {
             this.key = key;
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteBiTuple<Key, Key> toRange() {
+        @Override public IgniteBiTuple<ByteArray, ByteArray> toRange() {
             return new IgniteBiTuple<>(key, key);
         }
 
         /** {@inheritDoc} */
-        @Override public boolean contains(Key key) {
+        @Override public boolean contains(ByteArray key) {
             return this.key.equals(key);
         }
 
@@ -74,10 +74,10 @@ public interface KeyCriterion {
      */
     static class RangeCriterion implements KeyCriterion {
         /** Start of the range. */
-        private final Key from;
+        private final ByteArray from;
 
         /** End of the range (exclusive). */
-        private final Key to;
+        private final ByteArray to;
 
         /**
          * Creates the instance of range criterion.
@@ -85,18 +85,18 @@ public interface KeyCriterion {
          * @param from Start of the range.
          * @param to End of the range (exclusive).
          */
-        public RangeCriterion(Key from, Key to) {
+        public RangeCriterion(ByteArray from, ByteArray to) {
             this.from = from;
             this.to = to;
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteBiTuple<Key, Key> toRange() {
+        @Override public IgniteBiTuple<ByteArray, ByteArray> toRange() {
             return new IgniteBiTuple<>(from, to);
         }
 
         /** {@inheritDoc} */
-        @Override public boolean contains(Key key) {
+        @Override public boolean contains(ByteArray key) {
             return key.compareTo(from) >= 0 && key.compareTo(to) < 0;
         }
     }
@@ -106,24 +106,24 @@ public interface KeyCriterion {
      */
     static class CollectionCriterion implements KeyCriterion {
         /** Collection of keys. */
-        private final Collection<Key> keys;
+        private final Collection<ByteArray> keys;
 
         /**
          * Creates the instance of collection criterion.
          *
          * @param keys Collection of keys.
          */
-        public CollectionCriterion(Collection<Key> keys) {
+        public CollectionCriterion(Collection<ByteArray> keys) {
             this.keys = keys;
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteBiTuple<Key, Key> toRange() {
+        @Override public IgniteBiTuple<ByteArray, ByteArray> toRange() {
             return new IgniteBiTuple<>(Collections.min(keys), Collections.max(keys));
         }
 
         /** {@inheritDoc} */
-        @Override public boolean contains(Key key) {
+        @Override public boolean contains(ByteArray key) {
             return keys.contains(key);
         }
     }
@@ -133,29 +133,31 @@ public interface KeyCriterion {
      */
     static class PrefixCriterion implements KeyCriterion {
         /** Prefix of the key. */
-        private final Key prefixKey;
+        private final ByteArray prefixKey;
 
         /**
          * Creates the instance of prefix key criterion.
          *
          * @param prefixKey Prefix of the key.
          */
-        public PrefixCriterion(Key prefixKey) {
+        public PrefixCriterion(ByteArray prefixKey) {
             this.prefixKey = prefixKey;
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteBiTuple<Key, Key> toRange() {
+        @Override public IgniteBiTuple<ByteArray, ByteArray> toRange() {
             var bytes = Arrays.copyOf(prefixKey.bytes(), prefixKey.bytes().length);
+
             if (bytes[bytes.length - 1] != Byte.MAX_VALUE)
                 bytes[bytes.length - 1]++;
             else
                 bytes = Arrays.copyOf(bytes, bytes.length + 1);
-            return new IgniteBiTuple<>(prefixKey, new Key(bytes));
+
+            return new IgniteBiTuple<>(prefixKey, new ByteArray(bytes));
         }
 
         /** {@inheritDoc} */
-        @Override public boolean contains(Key key) {
+        @Override public boolean contains(ByteArray key) {
             return key.compareTo(prefixKey) >= 0 && key.compareTo(toRange().getValue()) < 0;
         }
     }
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
index 35c8881..6fb8023 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
@@ -25,10 +25,12 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiConsumer;
+
+import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.WatchEvent;
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.metastorage.client.EntryEvent;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -44,7 +46,7 @@ public class WatchAggregator {
     private final Map<Long, Watch> watches = Collections.synchronizedMap(new LinkedHashMap<>());
 
     /** Simple auto increment id for internal watches. */
-    private AtomicLong idCntr = new AtomicLong(0);
+    private final AtomicLong idCntr = new AtomicLong(0);
 
     /**
      * Adds new watch with simple exact criterion.
@@ -53,7 +55,7 @@ public class WatchAggregator {
      * @param lsnr Listener which will be executed on watch event.
      * @return id of registered watch. Can be used for remove watch from later.
      */
-    public long add(Key key, WatchListener lsnr) {
+    public long add(ByteArray key, WatchListener lsnr) {
         var watch = new Watch(new KeyCriterion.ExactCriterion(key), lsnr);
         var id = idCntr.incrementAndGet();
         watches.put(id, watch);
@@ -67,7 +69,7 @@ public class WatchAggregator {
      * @param lsnr Listener which will be executed on watch event.
      * @return id of registered watch. Can be used for remove watch from later.
      */
-    public long addPrefix(Key key, WatchListener lsnr) {
+    public long addPrefix(ByteArray key, WatchListener lsnr) {
         var watch = new Watch(new KeyCriterion.PrefixCriterion(key), lsnr);
         var id = idCntr.incrementAndGet();
         watches.put(id, watch);
@@ -81,7 +83,7 @@ public class WatchAggregator {
      * @param lsnr Listener which will be executed on watch event.
      * @return id of registered watch. Can be used for remove watch from later.
      */
-    public long add(Collection<Key> keys, WatchListener lsnr) {
+    public long add(Collection<ByteArray> keys, WatchListener lsnr) {
         var watch = new Watch(new KeyCriterion.CollectionCriterion(keys), lsnr);
         var id = idCntr.incrementAndGet();
         watches.put(id, watch);
@@ -96,7 +98,7 @@ public class WatchAggregator {
      * @param lsnr Listener which will be executed on watch event.
      * @return id of registered watch. Can be used for remove watch from later.
      */
-    public long add(Key from, Key to, WatchListener lsnr) {
+    public long add(ByteArray from, ByteArray to, WatchListener lsnr) {
         var watch = new Watch(new KeyCriterion.RangeCriterion(from, to), lsnr);
         var id = idCntr.incrementAndGet();
         watches.put(id, watch);
@@ -128,7 +130,10 @@ public class WatchAggregator {
      * @param saveRevisionAct action to commit keys-revision pair to persistent store for processed keys.
      * @return result aggregated watch.
      */
-    public Optional<AggregatedWatch> watch(long revision, BiConsumer<Collection<IgniteBiTuple<Key, byte[]>>, Long> saveRevisionAct) {
+    public Optional<AggregatedWatch> watch(
+            long revision,
+            BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> saveRevisionAct
+    ) {
         synchronized (watches) {
             if (watches.isEmpty())
                 return Optional.empty();
@@ -145,8 +150,8 @@ public class WatchAggregator {
     // TODO: IGNITE-14667 We can do it better than infer range always
     private KeyCriterion inferGeneralCriteria() {
         return new KeyCriterion.RangeCriterion(
-            watches.values().stream().map(w -> w.keyCriterion().toRange().getKey()).min(Key::compareTo).get(),
-            watches.values().stream().map(w -> w.keyCriterion().toRange().getValue()).max(Key::compareTo).get()
+            watches.values().stream().map(w -> w.keyCriterion().toRange().getKey()).min(ByteArray::compareTo).get(),
+            watches.values().stream().map(w -> w.keyCriterion().toRange().getValue()).max(ByteArray::compareTo).get()
         );
     }
 
@@ -156,7 +161,7 @@ public class WatchAggregator {
      * @param storeRevision action to commit keys-revision pair to persistent store for processed keys.
      * @return watch listener, which will dispatch events to appropriate watches.
      */
-    private WatchListener watchListener(BiConsumer<Collection<IgniteBiTuple<Key, byte[]>>, Long> storeRevision) {
+    private WatchListener watchListener(BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> storeRevision) {
         // Copy watches to separate collection, because all changes on the WatchAggregator watches
         // shouldn't be propagated to listener watches immediately.
         // WatchAggregator will be redeployed with new watches if needed instead.
@@ -164,22 +169,22 @@ public class WatchAggregator {
 
         return new WatchListener() {
 
-            @Override public boolean onUpdate(@NotNull Iterable<WatchEvent> evts) {
+            @Override public boolean onUpdate(@NotNull WatchEvent evt) {
                 var watchIt = cpWatches.entrySet().iterator();
                 Collection<Long> toCancel = new ArrayList<>();
 
                 while (watchIt.hasNext()) {
                     Map.Entry<Long, WatchAggregator.Watch> entry = watchIt.next();
                     WatchAggregator.Watch watch = entry.getValue();
-                    var filteredEvts = new ArrayList<WatchEvent>();
+                    var filteredEvts = new ArrayList<EntryEvent>();
 
-                    for (WatchEvent evt : evts) {
-                        if (watch.keyCriterion().contains(evt.oldEntry().key()))
-                            filteredEvts.add(evt);
+                    for (EntryEvent entryEvt : evt.entryEvents()) {
+                        if (watch.keyCriterion().contains(entryEvt.oldEntry().key()))
+                            filteredEvts.add(entryEvt);
                     }
 
                     if (!filteredEvts.isEmpty()) {
-                        if (!watch.listener().onUpdate(filteredEvts)) {
+                        if (!watch.listener().onUpdate(new WatchEvent(filteredEvts))) {
                             watchIt.remove();
 
                             toCancel.add(entry.getKey());
@@ -193,11 +198,11 @@ public class WatchAggregator {
                     cancelAll(toCancel);
 
                 var revision = 0L;
-                var entries = new ArrayList<IgniteBiTuple<Key, byte[]>>();
-                for (WatchEvent evt: evts) {
-                    revision = evt.newEntry().revision();
+                var entries = new ArrayList<IgniteBiTuple<ByteArray, byte[]>>();
+                for (EntryEvent entryEvt: evt.entryEvents()) {
+                    revision = entryEvt.newEntry().revision();
 
-                    entries.add(new IgniteBiTuple<>(evt.newEntry().key(), evt.newEntry().value()));
+                    entries.add(new IgniteBiTuple<>(entryEvt.newEntry().key(), entryEvt.newEntry().value()));
                 }
 
                 storeRevision.accept(entries, revision);
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
index 1360668..b50d603 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
@@ -18,17 +18,16 @@
 package org.apache.ignite.internal.metastorage;
 
 import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.Collections;
+import java.util.List;
 import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
-import org.apache.ignite.metastorage.common.Entry;
-import org.apache.ignite.metastorage.common.Key;
-import org.apache.ignite.metastorage.common.WatchEvent;
-import org.apache.ignite.metastorage.common.WatchListener;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.metastorage.client.Entry;
+import org.apache.ignite.metastorage.client.EntryEvent;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.Test;
-
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -42,19 +41,27 @@ public class WatchAggregatorTest {
         var watchAggregator = new WatchAggregator();
         var lsnr1 = mock(WatchListener.class);
         var lsnr2 = mock(WatchListener.class);
-        watchAggregator.add(new Key("1"), lsnr1);
-        watchAggregator.add(new Key("2"), lsnr2);
+        watchAggregator.add(new ByteArray("1"), lsnr1);
+        watchAggregator.add(new ByteArray("2"), lsnr2);
 
-        var watchEvent1 = new WatchEvent(
-            entry("1", "value1", 1, 1),
-            entry("1", "value1n", 1, 1));
-        var watchEvent2 = new WatchEvent(
-            entry("2", "value2", 1, 1),
-            entry("2", "value2n", 1, 1));
-        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(Arrays.asList(watchEvent1, watchEvent2));
+        var entryEvt1 = new EntryEvent(
+                entry("1", "value1", 1, 1),
+                entry("1", "value1n", 1, 1)
+        );
+
+        var watchEvent1 = new WatchEvent(entryEvt1);
+
+        var entryEvt2 = new EntryEvent(
+                entry("2", "value2", 1, 1),
+                entry("2", "value2n", 1, 1)
+        );
+
+        var watchEvent2 = new WatchEvent(entryEvt2);
+
+        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
-        verify(lsnr1).onUpdate(Collections.singletonList(watchEvent1));
-        verify(lsnr2).onUpdate(Collections.singletonList(watchEvent2));
+        verify(lsnr1).onUpdate(watchEvent1);
+        verify(lsnr2).onUpdate(watchEvent2);
     }
 
     @Test
@@ -64,22 +71,26 @@ public class WatchAggregatorTest {
         when(lsnr1.onUpdate(any())).thenReturn(true);
         var lsnr2 = mock(WatchListener.class);
         when(lsnr2.onUpdate(any())).thenReturn(true);
-        var id1 = watchAggregator.add(new Key("1"), lsnr1);
-        var id2 = watchAggregator.add(new Key("2"), lsnr2);
+        var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
+        var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
 
-        var watchEvent1 = new WatchEvent(
-            entry("1", "value1", 1, 1),
-            entry("1", "value1n", 1, 1));
-        var watchEvent2 = new WatchEvent(
-            entry("2", "value2", 1, 1),
-            entry("2", "value2n", 1, 1));
-        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(Arrays.asList(watchEvent1, watchEvent2));
+        var entryEvt1 = new EntryEvent(
+                entry("1", "value1", 1, 1),
+                entry("1", "value1n", 1, 1)
+        );
+
+        var entryEvt2 = new EntryEvent(
+                entry("2", "value2", 1, 1),
+                entry("2", "value2n", 1, 1)
+        );
+
+        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
         verify(lsnr1, times(1)).onUpdate(any());
         verify(lsnr2, times(1)).onUpdate(any());
 
         watchAggregator.cancel(id1);
-        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(Arrays.asList(watchEvent1, watchEvent2));
+        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
         verify(lsnr1, times(1)).onUpdate(any());
         verify(lsnr2, times(2)).onUpdate(any());
@@ -92,21 +103,25 @@ public class WatchAggregatorTest {
         when(lsnr1.onUpdate(any())).thenReturn(false);
         var lsnr2 = mock(WatchListener.class);
         when(lsnr2.onUpdate(any())).thenReturn(true);
-        var id1 = watchAggregator.add(new Key("1"), lsnr1);
-        var id2 = watchAggregator.add(new Key("2"), lsnr2);
+        var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
+        var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
 
-        var watchEvent1 = new WatchEvent(
+        var entryEvt1 = new EntryEvent(
             entry("1", "value1", 1, 1),
-            entry("1", "value1n", 1, 1));
-        var watchEvent2 = new WatchEvent(
+            entry("1", "value1n", 1, 1)
+        );
+
+        var entryEvt2 = new EntryEvent(
             entry("2", "value2", 1, 1),
-            entry("2", "value2n", 1, 1));
-        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(Arrays.asList(watchEvent1, watchEvent2));
+            entry("2", "value2n", 1, 1)
+        );
+
+        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
         verify(lsnr1, times(1)).onUpdate(any());
         verify(lsnr2, times(1)).onUpdate(any());
 
-        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(Arrays.asList(watchEvent1, watchEvent2));
+        watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
         verify(lsnr1, times(1)).onUpdate(any());
         verify(lsnr2, times(2)).onUpdate(any());
@@ -115,8 +130,8 @@ public class WatchAggregatorTest {
 
     private Entry entry(String key, String value, long revision, long updateCntr) {
         return new Entry() {
-            @Override public @NotNull Key key() {
-                return new Key(key);
+            @Override public @NotNull ByteArray key() {
+                return new ByteArray(key);
             }
 
             @Override public @Nullable byte[] value() {
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
index d35524b..cf237a4 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.stream.Collectors;
 import org.apache.ignite.app.Ignite;
 import org.apache.ignite.app.Ignition;
 import org.apache.ignite.configuration.RootKey;
@@ -192,9 +191,7 @@ public class IgnitionImpl implements Ignition {
     private static void ackBanner() {
         String ver = IgniteProperties.get(VER_KEY);
 
-        String banner = Arrays
-            .stream(BANNER)
-            .collect(Collectors.joining("\n"));
+        String banner = String.join("\n", BANNER);
 
         LOG.info(banner + '\n' + " ".repeat(22) + "Apache Ignite ver. " + ver + '\n');
     }
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/storage/DistributedConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/storage/DistributedConfigurationStorage.java
index bd76c33..2432fd6 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/storage/DistributedConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/storage/DistributedConfigurationStorage.java
@@ -35,39 +35,41 @@ import org.apache.ignite.configuration.storage.Data;
 import org.apache.ignite.configuration.storage.StorageException;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+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.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.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.Entry;
+import org.apache.ignite.metastorage.client.EntryEvent;
+import org.apache.ignite.metastorage.client.Operation;
+import org.apache.ignite.metastorage.client.Operations;
+import org.apache.ignite.metastorage.client.WatchEvent;
+import org.apache.ignite.metastorage.client.WatchListener;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * Distributed configuration storage.
  */
 public class DistributedConfigurationStorage implements ConfigurationStorage {
-    /** Prefix that we add to configuration keys to distinguish them in metastorage. Must end with dot. */
+    /** Prefix that we add to configuration keys to distinguish them in meta storage. Must end with dot. */
     private static final String DISTRIBUTED_PREFIX = "dst-cfg.";
 
     /** Logger. */
     private static final IgniteLogger LOG = IgniteLogger.forClass(DistributedConfigurationStorage.class);
 
     /**
-     * Key for CAS-ing configuration keys to metastorage. This key is expected to be the first key in lexicographical
+     * Key for CAS-ing configuration keys to meta storage. This key is expected to be the first key in lexicographical
      * order of distributed configuration keys.
      */
-    private static final Key MASTER_KEY = new Key(DISTRIBUTED_PREFIX);
+    private static final ByteArray MASTER_KEY = new ByteArray(DISTRIBUTED_PREFIX);
 
     /**
      * This key is expected to be the last key in lexicographical order of distributed configuration keys. It is
-     * possible because keys are in lexicographical order in metastorage and adding {@code (char)('.' + 1)} to the end
+     * possible because keys are in lexicographical order in meta storage and adding {@code (char)('.' + 1)} to the end
      * will produce all keys with prefix {@link DistributedConfigurationStorage#DISTRIBUTED_PREFIX}
      */
-    private static final Key DST_KEYS_END_RANGE = new Key(DISTRIBUTED_PREFIX.substring(0, DISTRIBUTED_PREFIX.length() - 1) + (char)('.' + 1));
+    private static final ByteArray DST_KEYS_END_RANGE =
+            new ByteArray(DISTRIBUTED_PREFIX.substring(0, DISTRIBUTED_PREFIX.length() - 1) + (char)('.' + 1));
 
     /** Id of watch that is responsible for configuration update. */
     private CompletableFuture<Long> watchId;
@@ -76,10 +78,10 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
     private final MetaStorageManager metaStorageMgr;
 
     /** Change listeners. */
-    private List<ConfigurationStorageListener> listeners = new CopyOnWriteArrayList<>();
+    private final List<ConfigurationStorageListener> listeners = new CopyOnWriteArrayList<>();
 
-    /** Storage version. It stores actual metastorage revision, that is applied to configuration manager. */
-    private AtomicLong ver = new AtomicLong(0L);
+    /** Storage version. It stores actual meta storage revision, that is applied to configuration manager. */
+    private final AtomicLong ver = new AtomicLong(0L);
 
     /**
      * Constructor.
@@ -141,7 +143,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
         HashSet<Operation> operations = new HashSet<>();
 
         for (Map.Entry<String, Serializable> entry : newValues.entrySet()) {
-            Key key = new Key(DISTRIBUTED_PREFIX + entry.getKey());
+            ByteArray key = new ByteArray(DISTRIBUTED_PREFIX + entry.getKey());
 
             if (entry.getValue() != null)
                 // TODO: investigate overhead when serialize int, long, double, boolean, string, arrays of above
@@ -154,7 +156,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
         operations.add(Operations.put(MASTER_KEY, ByteUtils.longToBytes(sentVersion)));
 
         return metaStorageMgr.invoke(
-            Conditions.key(MASTER_KEY).revision().eq(ver.get()),
+            Conditions.revision(MASTER_KEY).eq(ver.get()),
             operations,
             Collections.singleton(Operations.noop()));
     }
@@ -167,14 +169,14 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
             // TODO: registerWatchByPrefix could throw OperationTimeoutException and CompactedException and we should
             // TODO: properly handle such cases https://issues.apache.org/jira/browse/IGNITE-14604
             watchId = metaStorageMgr.registerWatchByPrefix(MASTER_KEY, new WatchListener() {
-                @Override public boolean onUpdate(@NotNull Iterable<WatchEvent> events) {
+                @Override public boolean onUpdate(@NotNull WatchEvent events) {
                     HashMap<String, Serializable> data = new HashMap<>();
 
                     long maxRevision = 0L;
 
                     Entry entryForMasterKey = null;
 
-                    for (WatchEvent event : events) {
+                    for (EntryEvent event : events.entryEvents()) {
                         Entry e = event.newEntry();
 
                         if (!e.key().equals(MASTER_KEY)) {
@@ -187,7 +189,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
                             entryForMasterKey = e;
                     }
 
-                    // Contract of metastorage ensures that all updates of one revision will come in one batch.
+                    // Contract of meta storage ensures that all updates of one revision will come in one batch.
                     // Also masterKey should be updated every time when we update cfg.
                     // That means that masterKey update must be included in the batch.
                     assert entryForMasterKey != null;
@@ -206,7 +208,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
                 @Override public void onError(@NotNull Throwable e) {
                     // TODO: need to handle this case and there should some mechanism for registering new watch as far as
                     // TODO: onError unregisters failed watch https://issues.apache.org/jira/browse/IGNITE-14604
-                    LOG.error("Metastorage listener issue", e);
+                    LOG.error("Meta storage listener issue", e);
                 }
             });
         }
@@ -245,11 +247,11 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
     }
 
     /**
-     * Method that returns all distributed configuration keys from metastorage filtered out by the current applied
+     * Method that returns all distributed configuration keys from meta storage filtered out by the current applied
      * revision as an upper bound. Applied revision is a revision of the last successful vault update.
      * <p>
-     * This is possible to distinguish cfg keys from metastorage because we add special prefix {@link
-     * DistributedConfigurationStorage#DISTRIBUTED_PREFIX} to all configuration keys that we put to metastorage.
+     * This is possible to distinguish cfg keys from meta storage because we add special prefix {@link
+     * DistributedConfigurationStorage#DISTRIBUTED_PREFIX} to all configuration keys that we put to meta storage.
      *
      * @return Cursor built upon all distributed configuration entries.
      */
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index a5c1cc0..02ca745 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -47,10 +47,13 @@ import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
 import org.apache.ignite.internal.table.event.TableEvent;
 import org.apache.ignite.internal.table.event.TableEventParameters;
 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.client.Conditions;
+import org.apache.ignite.metastorage.client.Key;
+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.apache.ignite.raft.client.service.RaftGroupService;
 import org.apache.ignite.table.Table;
@@ -83,7 +86,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private final AffinityManager affMgr;
 
     /** Tables. */
-    private Map<String, TableImpl> tables = new ConcurrentHashMap<>();
+    private final Map<String, TableImpl> tables = new ConcurrentHashMap<>();
 
     /**
      * Creates a new table manager.
@@ -199,10 +202,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                 UUID tblId = new UUID(revision, 0L);
 
                 if (hasMetastorageLocally) {
-                    var key = new Key(INTERNAL_PREFIX + tblId.toString());
+                    var key = new ByteArray(INTERNAL_PREFIX + tblId.);
 
                     futs.add(metaStorageMgr.invoke(
-                        Conditions.key(key).value().eq(null),
+                        Conditions.exists(key),
                         Operations.put(key, tableView.name().getBytes(StandardCharsets.UTF_8)),
                         Operations.noop()).thenCompose(res ->
                         affMgr.calculateAssignments(tblId)));
@@ -240,10 +243,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                 UUID tblId = t.internalTable().tableId();
 
                 if (hasMetastorageLocally) {
-                    var key = new Key(INTERNAL_PREFIX + tblId.toString());
+                    var key = new ByteArray(INTERNAL_PREFIX + tblId.toString());
 
                     futs.add(affMgr.removeAssignment(tblId).thenCompose(res ->
-                        metaStorageMgr.invoke(Conditions.key(key).value().ne(null),
+                        metaStorageMgr.invoke(Conditions.exists(key),
                             Operations.remove(key),
                             Operations.noop())));
                 }

[ignite-3] 06/09: IGNITE-14389 Added putAll and removeAll. Started cursor management: ranges and watches (WIP)

Posted by ag...@apache.org.
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 efe4f16c23f3e061a7f43089092e5165a2fc3b5e
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Fri Apr 23 02:35:28 2021 +0300

    IGNITE-14389 Added putAll and removeAll. Started cursor management: ranges and watches (WIP)
---
 .../metastorage/server/KeyValueStorage.java        |  10 +
 .../server/SimpleInMemoryKeyValueStorage.java      | 186 +++++++++-
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 380 +++++++++++++++++++++
 3 files changed, 565 insertions(+), 11 deletions(-)

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
index 0f18ece..526e4fb 100644
--- 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
@@ -31,11 +31,21 @@ public interface KeyValueStorage {
 
     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);
+
+    Iterator<Entry> range(byte[] keyFrom, byte[] keyTo);
+
     Iterator<Entry> iterate(byte[] key);
 
     //Iterator<Entry> iterate(long rev);
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
index f532005..32f720e 100644
--- 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
@@ -1,15 +1,9 @@
 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.*;
+import java.util.function.Consumer;
+
+import org.apache.ignite.metastorage.common.Cursor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.TestOnly;
 
@@ -25,6 +19,8 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
     private final Watcher watcher;
 
+    private final List<Cursor<Entry>> rangeCursors = new ArrayList<>();
+
     private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
 
     private NavigableMap<Long, NavigableMap<byte[], Value>> revsIdx = new TreeMap<>();
@@ -66,8 +62,25 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     @Override
     public void putAll(List<byte[]> keys, List<byte[]> values) {
         synchronized (mux) {
+            long curRev = rev + 1;
+
+            doPutAll(curRev, keys, values);
+        }
+    }
+
+    @Override
+    public @NotNull 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;
     }
 
     @NotNull
@@ -119,6 +132,69 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         }
     }
 
+    @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 (int i = 0; i < keys.size(); i++) {
+                byte[] key = keys.get(i);
+
+                Entry e = doGet(key, LATEST_REV, false);
+
+                if (e.empty() || e.tombstone())
+                    continue;
+
+                existingKeys.add(key);
+
+                vals.add(TOMBSTONE);
+            }
+
+            doPutAll(curRev, existingKeys, vals);
+        }
+    }
+
+    @Override
+    public @NotNull 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 (int i = 0; i < keys.size(); i++) {
+                byte[] key = keys.get(i);
+
+                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;
+    }
+
+    @Override
+    public Iterator<Entry> range(byte[] keyFrom, byte[] keyTo) {
+        return null;
+    }
+
     @Override public Iterator<Entry> iterate(byte[] keyFrom) {
         synchronized (mux) {
             NavigableMap<byte[], List<Long>> tailMap = keysIdx.tailMap(keyFrom, true);
@@ -237,7 +313,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     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 : "Revision must be positive.";
+        assert rev > 0 || rev == LATEST_REV: "Revision must be positive.";
 
         Collection<Entry> res = new ArrayList<>(keys.size());
 
@@ -344,6 +420,39 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return lastRev;
     }
 
+    private long doPutAll(long curRev, List<byte[]> keys, List<byte[]> bytesList) {
+        synchronized (mux) {
+            // Update revsIdx.
+            NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+            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<>());
+
+                long lastRev = revs.isEmpty() ? 0 : lastRevision(revs);
+
+                revs.add(curRev);
+
+                Value val = new Value(bytes, curUpdCntr);
+
+                entries.put(key, val);
+
+                revsIdx.put(curRev, entries);
+            }
+
+            rev = curRev;
+
+            return curRev;
+        }
+    }
+
+
     private static boolean isPrefix(byte[] pref, byte[] term) {
         if (pref.length > term.length)
             return false;
@@ -368,4 +477,59 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         return res;
     }
 
+    private class RangeCursor implements Cursor<Entry> {
+        private final byte[] keyFrom;
+        private final byte[] keyTo;
+        private final long rev;
+        private byte[] curKey;
+
+        public RangeCursor(byte[] keyFrom, byte[] keyTo, long rev) {
+            this.keyFrom = keyFrom;
+            this.keyTo = keyTo;
+            this.rev = rev;
+        }
+
+        @Override public void close() throws Exception {
+
+        }
+
+        @NotNull
+        @Override public Iterator<Entry> iterator() {
+            return new Iterator<Entry>() {
+                @Override public boolean hasNext() {
+                    synchronized (mux) {
+                        byte[] key = keysIdx.ceilingKey(curKey);
+
+                        return key != null;
+                    }
+                }
+
+                @Override public Entry next() {
+                    synchronized (mux) {
+                        Map.Entry<byte[], List<Long>> e = keysIdx.ceilingEntry(curKey);
+
+                        if (e == null)
+                            throw new NoSuchElementException();
+
+                        List<Long> revs = e.getValue();
+
+                        assert revs != null && !revs.isEmpty() :
+                                "Revisions should not be empty: [revs=" + revs + ']';
+
+                        //lastRevision(re)
+
+                        return null;
+                    }
+                }
+            };
+        }
+
+        @Override public void forEach(Consumer<? super Entry> action) {
+            Cursor.super.forEach(action);
+        }
+
+        @Override public Spliterator<Entry> spliterator() {
+            return Cursor.super.spliterator();
+        }
+    }
 }
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
index fa130e6..4a73137 100644
--- 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
@@ -272,6 +272,191 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(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 Key(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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(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 Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        e1 = map.get(new Key(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 Key(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 Key(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 Key(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
     public void remove() {
         byte[] key = k(1);
         byte[] val = kv(1, 1);
@@ -377,6 +562,201 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(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 Key(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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(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 Key(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 Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        e1 = map.get(new Key(key1));
+
+        assertNotNull(e1);
+        assertEquals(6, e1.revision());
+        assertEquals(6, e1.updateCounter());
+        assertTrue(e1.tombstone());
+        assertFalse(e1.empty());
+
+        // Test rewritten value.
+        e2 = map.get(new Key(key2));
+
+        assertNotNull(e2);
+        assertEquals(6, e2.revision());
+        assertEquals(7, e2.updateCounter());
+        assertTrue(e2.tombstone());
+        assertFalse(e2.empty());
+
+        // Test removed value.
+        e3 = map.get(new Key(key3));
+
+        assertNotNull(e3);
+        assertEquals(5, e3.revision());
+        assertEquals(5, e3.updateCounter());
+        assertTrue(e3.tombstone());
+        assertFalse(e3.empty());
+
+        // Test empty value.
+        e4 = map.get(new Key(key4));
+
+        assertNotNull(e4);
+        assertFalse(e4.tombstone());
+        assertTrue(e4.empty());
+    }
+
+    @Test
     public void getAfterRemove() {
         byte[] key = k(1);
         byte[] val = kv(1, 1);

[ignite-3] 01/09: IGNITE-14389 Meta storage: in-memory implementation WIP

Posted by ag...@apache.org.
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 a1eb463e8bcba21f50e3b4ddd6996944d9d95d20
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Tue Mar 30 21:21:07 2021 +0300

    IGNITE-14389 Meta storage: in-memory implementation WIP
---
 modules/metastorage-server/pom.xml                 |  60 +++++
 .../ignite/internal/metastorage/server/Entry.java  | 146 +++++++++++
 .../metastorage/server/KeyValueStorage.java        |  28 +++
 .../server/SimpleInMemoryKeyValueStorage.java      | 267 ++++++++++++++++++++
 .../ignite/internal/metastorage/server/Watch.java  |  45 ++++
 .../internal/metastorage/server/Watcher.java       |  13 +
 .../internal/metastorage/server/WatcherImpl.java   |  58 +++++
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 274 +++++++++++++++++++++
 pom.xml                                            |   1 +
 9 files changed, 892 insertions(+)

diff --git a/modules/metastorage-server/pom.xml b/modules/metastorage-server/pom.xml
new file mode 100644
index 0000000..3c51fc5
--- /dev/null
+++ b/modules/metastorage-server/pom.xml
@@ -0,0 +1,60 @@
+<?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>metastorage-server</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </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-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..442aef9
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Entry.java
@@ -0,0 +1,146 @@
+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>
+ *     <ul>value - a data which is associated with a key and represented as an array of bytes.</ul>
+ *     <ul>revision - a number which denotes a version of whole meta storage. Each change increments the revision.</ul>
+ * </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>
+ */
+//TODO: Separate client and server entries. Empty and tombstone for client is the same.
+public class Entry {
+    /** Entry key. Couldn't be {@code null}. */
+    @NotNull
+    final private byte[] key;
+
+    /**
+     * Entry value.
+     * <p>
+     *     {@code val == null} only for {@link #empty()} and {@link #tombstone()} entries.
+     * </p>
+     */
+    @Nullable
+    final private 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>
+     */
+    final private long rev;
+
+    /**
+     * Constructor.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @param val Value bytes. Couldn't be {@code null}.
+     * @param rev Revision.
+     */
+    // 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) {
+        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;
+    }
+
+    /**
+     * Constructor for empty and tombstone entries.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @param rev Revision.
+     */
+    private Entry(@NotNull byte[] key, long rev) {
+        assert key != null : "key can't be null";
+
+        this.key = key;
+        this.val = null;
+        this.rev = rev;
+    }
+
+    /**
+     * 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);
+    }
+
+    /**
+     * Creates an instance of tombstone entry for a given key and a revision.
+     *
+     * @param key Key bytes. Couldn't be {@code null}.
+     * @return Empty entry.
+     */
+    @NotNull
+    public static Entry tombstone(byte[] key, long rev) {
+        assert rev > 0 : "rev must be positive for tombstone entry.";
+
+        return new Entry(key, rev);
+    }
+
+    /**
+     * 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 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;
+    }
+
+    /**
+     * 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;
+    }
+}
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..1bf6b78
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
@@ -0,0 +1,28 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.NotNull;
+
+import java.util.Iterator;
+
+public interface KeyValueStorage {
+
+    long revision();
+
+    @NotNull
+    Entry put(byte[] key, byte[] value);
+
+    @NotNull
+    Entry get(byte[] key);
+
+    @NotNull
+    Entry get(byte[] key, long rev);
+
+    @NotNull
+    Entry remove(byte[] key);
+
+    Iterator<Entry> iterate(byte[] key);
+
+    //Iterator<Entry> iterate(long rev);
+
+    void compact();
+}
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..9059aec
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -0,0 +1,267 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+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 org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.TestOnly;
+
+/**
+ * WARNING: Only for test purposes and only for non-distributed (one static instance) storage.
+ */
+public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
+    private static final Comparator<byte[]> LEXICOGRAPHIC_COMPARATOR = Arrays::compare;
+
+    private static final byte[] TOMBSTONE = new byte[0];
+
+    private static final long LATEST_REV = -1;
+
+    private final Watcher watcher;
+
+    private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+    private NavigableMap<Long, NavigableMap<byte[], byte[]>> revsIdx = new TreeMap<>();
+
+    private long grev = 0;
+
+    private final Object mux = new Object();
+
+    public SimpleInMemoryKeyValueStorage(Watcher watcher) {
+        this.watcher = watcher;
+    }
+
+    @Override public long revision() {
+        return grev;
+    }
+
+    @NotNull
+    @Override public Entry put(byte[] key, byte[] val) {
+        synchronized (mux) {
+            long crev = ++grev;
+
+            // Update keysIdx.
+            List<Long> revs = keysIdx.computeIfAbsent(key, k -> new ArrayList<>());
+
+            long lrev = revs.isEmpty() ? 0 : lastRevision(revs);
+
+            revs.add(crev);
+
+            // Update revsIdx.
+            NavigableMap<byte[], byte[]> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+            entries.put(key, val);
+
+            revsIdx.put(crev, entries);
+
+            // Return previous value.
+            if (lrev == 0)
+                return Entry.empty(key);
+
+            NavigableMap<byte[], byte[]> lastVal = revsIdx.get(lrev);
+
+            Entry res = new Entry(key, lastVal.get(key), lrev);
+
+            //TODO: notify watchers
+
+            return res;
+        }
+    }
+
+    @NotNull
+    @Override public Entry get(byte[] key) {
+        synchronized (mux) {
+            return doGet(key, LATEST_REV);
+        }
+    }
+
+    @NotNull
+    @TestOnly
+    @Override public Entry get(byte[] key, long rev) {
+        synchronized (mux) {
+            return doGet(key, rev);
+        }
+    }
+
+    @NotNull
+    @Override public Entry remove(byte[] key) {
+        synchronized (mux) {
+            Entry e = doGet(key, LATEST_REV);
+
+            if (e.value() == null)
+                return e;
+
+            return put(key, TOMBSTONE);
+        }
+    }
+
+    @Override public Iterator<Entry> iterate(byte[] keyFrom) {
+        synchronized (mux) {
+            NavigableMap<byte[], List<Long>> tailMap = keysIdx.tailMap(keyFrom, true);
+
+            final Iterator<Map.Entry<byte[], List<Long>>> it = tailMap.entrySet().iterator();
+
+            return new Iterator<>() {
+                private Map.Entry<byte[], List<Long>> curr;
+                private boolean hasNext;
+
+                private void advance() {
+                    if (it.hasNext()) {
+                        Map.Entry<byte[], List<Long>> e = it.next();
+
+                        byte[] key = e.getKey();
+
+                        if (!isPrefix(keyFrom, key))
+                            hasNext = false;
+                        else {
+                            curr = e;
+
+                            hasNext = true;
+                        }
+                    } else
+                        hasNext = false;
+                }
+
+                @Override
+                public boolean hasNext() {
+                    synchronized (mux) {
+                        if (curr == null)
+                            advance();
+
+                        return hasNext;
+                    }
+                }
+
+                @Override
+                public Entry next() {
+                    synchronized (mux) {
+                        if (!hasNext())
+                            throw new NoSuchElementException();
+
+                        Map.Entry<byte[], List<Long>> e = curr;
+
+                        curr = null;
+
+                        byte[] key = e.getKey();
+
+                        List<Long> revs = e.getValue();
+
+                        long rev = revs == null || revs.isEmpty() ? 0 : lastRevision(revs);
+
+                        if (rev == 0) {
+                            throw new IllegalStateException("rev == 0");
+                            //return new AbstractMap.SimpleImmutableEntry<>(key, null);
+                        }
+
+                        NavigableMap<byte[], byte[]> vals = revsIdx.get(rev);
+
+                        if (vals == null || vals.isEmpty()) {
+                            throw new IllegalStateException("vals == null || vals.isEmpty()");
+                            //return new AbstractMap.SimpleImmutableEntry<>(key, null);
+                        }
+
+                        byte[] val = vals.get(key);
+
+                        return val == TOMBSTONE ? Entry.tombstone(key, rev) : new Entry(key, val, rev);
+                    }
+                }
+            };
+        }
+    }
+
+    @Override public void compact() {
+        synchronized (mux) {
+            NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+            NavigableMap<Long, NavigableMap<byte[], byte[]>> compactedRevsIdx = new TreeMap<>();
+
+            keysIdx.forEach((key, revs) -> compactForKey(key, revs, compactedKeysIdx, compactedRevsIdx));
+
+            keysIdx = compactedKeysIdx;
+
+            revsIdx = compactedRevsIdx;
+        }
+    }
+
+    private void compactForKey(
+            byte[] key,
+            List<Long> revs,
+            NavigableMap<byte[], List<Long>> compactedKeysIdx,
+            NavigableMap<Long, NavigableMap<byte[], byte[]>> compactedRevsIdx
+    ) {
+        Long lrev = lastRevision(revs);
+
+        NavigableMap<byte[], byte[]> kv = revsIdx.get(lrev);
+
+        byte[] lastVal = kv.get(key);
+
+        if (lastVal != TOMBSTONE) {
+            compactedKeysIdx.put(key, listOf(lrev));
+
+            NavigableMap<byte[], byte[]> compactedKv = compactedRevsIdx.computeIfAbsent(
+                    lrev,
+                    k -> new TreeMap<>(LEXICOGRAPHIC_COMPARATOR)
+            );
+
+            compactedKv.put(key, lastVal);
+        }
+    }
+
+    /**
+     * Returns entry for given key.
+     *
+     * @param key Key.
+     * @param rev Revision.
+     * @return Entry for given key.
+     */
+    @NotNull private Entry doGet(byte[] key, long rev) {
+        List<Long> revs = keysIdx.get(key);
+
+        if (revs == null || revs.isEmpty())
+            return Entry.empty(key);
+
+        long lrev = rev == LATEST_REV ? lastRevision(revs) : rev;
+
+        NavigableMap<byte[], byte[]> entries = revsIdx.get(lrev);
+
+        if (entries == null || entries.isEmpty())
+            return Entry.empty(key);
+
+        byte[] val = entries.get(key);
+
+        if (val == TOMBSTONE)
+            return Entry.tombstone(key, lrev);
+
+        return new Entry(key, val , lrev);
+    }
+
+    private static boolean isPrefix(byte[] pref, byte[] term) {
+        if (pref.length > term.length)
+            return false;
+
+        for (int i = 0; i < pref.length - 1; i++) {
+            if (pref[i] != term[i])
+                return false;
+        }
+
+        return true;
+    }
+
+    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;
+    }
+
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
new file mode 100644
index 0000000..26cfa5c
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
@@ -0,0 +1,45 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Arrays;
+import java.util.Comparator;
+
+public class Watch {
+    private static final Comparator<byte[]> CMP = Arrays::compare;
+
+    private static final long ANY_REVISION = -1;
+
+    @Nullable
+    private byte[] startKey;
+
+    @Nullable
+    private byte[] endKey;
+
+    long rev = ANY_REVISION;
+
+    public void startKey(byte[] startKey) {
+        this.startKey = startKey;
+    }
+
+    public void endKey(byte[] endKey) {
+        this.endKey = endKey;
+    }
+
+    public void revision(long rev) {
+        this.rev = rev;
+    }
+
+    public void notify(Entry e) {
+        if (startKey != null && CMP.compare(e.key(), startKey) < 0)
+            return;
+
+        if (endKey != null && CMP.compare(e.key(), endKey) > 0)
+            return;
+
+        if (rev != ANY_REVISION && e.revision() <= rev)
+            return;
+
+        System.out.println("Entry: key=" + new String(e.key()) + ", value=" + new String(e.value()) + ", rev=" + e.revision());
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java
new file mode 100644
index 0000000..5516d06
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Watcher.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.NotNull;
+
+public interface Watcher {
+    void register(@NotNull Watch watch);
+
+    void notify(@NotNull Entry e);
+
+    //TODO: implement
+    void cancel(@NotNull Watch watch);
+}
+
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java
new file mode 100644
index 0000000..dc126a0
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/WatcherImpl.java
@@ -0,0 +1,58 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.NotNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+public class WatcherImpl implements Watcher {
+    private final BlockingQueue<Entry> queue = new LinkedBlockingQueue<>();
+
+    private final List<Watch> watches = new ArrayList<>();
+
+    private volatile boolean stop;
+
+    private final Object mux = new Object();
+
+    @Override public void register(@NotNull Watch watch) {
+        synchronized (mux) {
+            watches.add(watch);
+        }
+    }
+
+    @Override public void notify(@NotNull Entry e) {
+        queue.offer(e);
+    }
+
+    @Override
+    public void cancel(@NotNull Watch watch) {
+        throw new UnsupportedOperationException("Not implemented yet.");
+    }
+
+    public void shutdown() {
+        stop = true;
+    }
+
+    private class WatcherWorker implements Runnable {
+        @Override public void run() {
+            while (!stop) {
+                try {
+                    Entry e = queue.poll(100, TimeUnit.MILLISECONDS);
+
+                    if (e != null) {
+                        synchronized (mux) {
+                            watches.forEach(w -> w.notify(e));
+                        }
+                    }
+                }
+                catch (InterruptedException interruptedException) {
+                    // No-op.
+                }
+            }
+        }
+    }
+}
+
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..f7fb17e
--- /dev/null
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
@@ -0,0 +1,274 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+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.assertTrue;
+
+class SimpleInMemoryKeyValueStorageTest {
+    private KeyValueStorage storage;
+
+    @BeforeEach
+    public void setUp() {
+        storage = new SimpleInMemoryKeyValueStorage(new NoOpWatcher());
+    }
+
+    @Test
+    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());
+
+        // Previous entry is empty.
+        Entry emptyEntry = storage.put(key1, val1_1);
+
+        assertEquals(1, storage.revision());
+        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, storage.revision());
+
+        // Previous entry is empty.
+        emptyEntry = storage.put(key2, val2_2);
+
+        assertEquals(2, storage.revision());
+        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, storage.revision());
+
+        // Previous entry is not empty.
+        e1_1 = storage.put(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(3, storage.revision());
+
+        // 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, storage.revision());
+
+        // Remove existing entry.
+        Entry e2_2 = storage.remove(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(4, storage.revision()); // Storage revision is changed.
+
+        // Remove already removed entry.
+        Entry tombstoneEntry = storage.remove(key2);
+
+        assertFalse(tombstoneEntry.empty());
+        assertTrue(tombstoneEntry.tombstone());
+        assertEquals(4, storage.revision()); // Storage revision is not changed.
+
+        // Compact and check that tombstones are removed.
+        storage.compact();
+
+        assertEquals(4, storage.revision());
+        assertTrue(storage.remove(key2).empty());
+        assertTrue(storage.get(key2).empty());
+
+        // Remove existing entry.
+        e1_3 = storage.remove(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(5, storage.revision()); // Storage revision is changed.
+
+        // Remove already removed entry.
+        tombstoneEntry = storage.remove(key1);
+
+        assertFalse(tombstoneEntry.empty());
+        assertTrue(tombstoneEntry.tombstone());
+        assertEquals(5, storage.revision()); // // Storage revision is not changed.
+
+        // Compact and check that tombstones are removed.
+        storage.compact();
+
+        assertEquals(5, storage.revision());
+        assertTrue(storage.remove(key1).empty());
+        assertTrue(storage.get(key1).empty());
+    }
+
+    @Test
+    void compact() {
+        assertEquals(0, storage.revision());
+
+        // Compact empty.
+        storage.compact();
+
+        assertEquals(0, storage.revision());
+
+        // Compact non-empty.
+        fill(storage, 1, 1);
+
+        assertEquals(1, storage.revision());
+
+        fill(storage, 2, 2);
+
+        assertEquals(3, storage.revision());
+
+        fill(storage, 3, 3);
+
+        assertEquals(6, storage.revision());
+
+        storage.remove(k(3));
+
+        assertEquals(7, storage.revision());
+        assertTrue(storage.get(k(3)).tombstone());
+
+        storage.compact();
+
+        assertEquals(7, storage.revision());
+
+        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());
+
+        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());
+        assertEquals(3, e2.revision());
+
+        Entry e3 = storage.get(k(3));
+
+        assertTrue(e3.empty());
+        assertTrue(storage.get(k(3), 5).empty());
+        assertTrue(storage.get(k(3), 6).empty());
+        assertTrue(storage.get(k(3), 7).empty());
+    }
+
+    @Test
+    void iterate() {
+        TreeMap<String, String> expFooMap = new TreeMap<>();
+        TreeMap<String, String> expKeyMap = new TreeMap<>();
+        TreeMap<String, String> expZooMap = new TreeMap<>();
+
+        fill("foo", storage, expFooMap);
+        fill("key", storage, expKeyMap);
+        fill("zoo", storage, expZooMap);
+
+        assertEquals(300, storage.revision());
+
+        assertIterate("key", storage, expKeyMap);
+        assertIterate("zoo", storage, expZooMap);
+        assertIterate("foo", storage, expFooMap);
+    }
+
+    private void assertIterate(String pref,  KeyValueStorage storage, TreeMap<String, String> expMap) {
+        Iterator<Entry> it = storage.iterate((pref + "_").getBytes());
+        Iterator<Map.Entry<String, String>> expIt = expMap.entrySet().iterator();
+
+        // Order.
+        while (it.hasNext()) {
+            Entry entry = it.next();
+            Map.Entry<String, String> expEntry = expIt.next();
+
+            assertEquals(expEntry.getKey(), new String(entry.key()));
+            assertEquals(expEntry.getValue(), new String(entry.value()));
+        }
+
+        // Range boundaries.
+        it = storage.iterate((pref + '_').getBytes());
+
+        while (it.hasNext()) {
+            Entry entry = it.next();
+
+            assertTrue(expMap.containsKey(new String(entry.key())));
+        }
+    }
+
+    private static void fill(String pref, KeyValueStorage storage, TreeMap<String, String> expMap) {
+        for (int i = 0; i < 100; i++) {
+            String keyStr = pref + '_' + i;
+
+            String valStr = "val_" + i;
+
+            expMap.put(keyStr, valStr);
+
+            byte[] key = keyStr.getBytes();
+
+            byte[] val = valStr.getBytes();
+
+            storage.put(key, val);
+        }
+    }
+
+    private static void fill(KeyValueStorage storage, int keySuffix, int num) {
+        for (int i = 0; i < num; i++)
+            storage.put(k(keySuffix), kv(keySuffix, i + 1));
+    }
+
+    private static byte[] k(int k) {
+        return ("key" + k).getBytes();
+    }
+
+    private static byte[] kv(int k, int v) {
+        return ("key" + k + '_' + "val" + v).getBytes();
+    }
+
+    private static class NoOpWatcher implements Watcher {
+        @Override public void register(@NotNull Watch watch) {
+            // No-op.
+        }
+
+        @Override public void notify(@NotNull Entry e) {
+            // No-op.
+        }
+
+        @Override public void cancel(@NotNull Watch watch) {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 0f89644..3e7da2e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,6 +46,7 @@
         <module>modules/metastorage</module>
         <module>modules/metastorage-client</module>
         <module>modules/metastorage-common</module>
+        <module>modules/metastorage-server</module>
         <module>modules/network</module>
         <module>modules/raft</module>
         <module>modules/raft-client</module>

[ignite-3] 09/09: IGNITE-14389 MetaStorageService integration

Posted by ag...@apache.org.
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 51d3fb4f902bad06d23197a2cb78edacb91d4822
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Tue May 18 12:48:48 2021 +0300

    IGNITE-14389 MetaStorageService integration
---
 modules/metastorage-client/pom.xml                 |   6 +
 .../client/ITMetaStorageServiceTest.java           |   4 +-
 .../internal/metastorage/client/CursorImpl.java    |  14 +-
 .../internal/metastorage/client/EntryImpl.java}    |  78 +++++---
 .../metastorage/client/MetaStorageServiceImpl.java | 197 ++++++++++++++-----
 .../ignite/metastorage/client/Condition.java       | 185 ++++++++----------
 .../metastorage/client/MetaStorageService.java     |   8 +-
 .../ignite/metastorage/client/Operation.java       |  71 ++++++-
 .../internal/metastorage/common/ConditionType.java |  53 ++++++
 .../RemoveCommand.java => OperationType.java}      |  30 +--
 .../{GetCommand.java => ConditionInfo.java}        |  68 ++++---
 .../{RemoveCommand.java => ErrorResponse.java}     |  30 +--
 .../metastorage/common/command/GetAllCommand.java  |  25 ++-
 .../common/command/GetAndPutAllCommand.java        |  23 ++-
 .../common/command/GetAndPutCommand.java           |   6 +-
 .../common/command/GetAndRemoveAllCommand.java     |  18 +-
 .../common/command/GetAndRemoveCommand.java        |   6 +-
 .../metastorage/common/command/GetCommand.java     |  10 +-
 .../metastorage/common/command/InvokeCommand.java  |  76 ++++++++
 ...moveCommand.java => MultipleEntryResponse.java} |  29 +--
 .../{PutCommand.java => OperationInfo.java}        |  53 ++++--
 .../metastorage/common/command/PutAllCommand.java  |  44 ++++-
 .../metastorage/common/command/PutCommand.java     |   6 +-
 .../metastorage/common/command/RangeCommand.java   |  14 +-
 .../common/command/RemoveAllCommand.java           |  22 +--
 .../metastorage/common/command/RemoveCommand.java  |   6 +-
 .../common/command/SingleEntryResponse.java        |  92 +++++++++
 .../common/command/WatchExactKeysCommand.java      |  22 +--
 .../common/command/WatchRangeKeysCommand.java      |  15 +-
 .../{EntryEvent.java => CompactedException.java}   |  43 +++--
 .../ignite/internal/metastorage/server/Entry.java  |   1 -
 .../internal/metastorage/server/EntryEvent.java    |   2 +-
 .../metastorage/server/KeyValueStorage.java        | 187 +++++++++++++++---
 .../internal/metastorage/server/Operation.java     |  41 ++--
 .../server/SimpleInMemoryKeyValueStorage.java      |   2 -
 .../metastorage/server/ValueCondition.java         |   2 +-
 .../server/raft/MetaStorageCommandListener.java    | 211 +++++++++++++++------
 .../server/SimpleInMemoryKeyValueStorageTest.java  |  69 +++----
 modules/metastorage/pom.xml                        |   4 +-
 .../internal/metastorage/MetaStorageManager.java   |  17 +-
 .../internal/metastorage/WatchAggregatorTest.java  |  15 +-
 .../internal/table/distributed/TableManager.java   |   2 +-
 parent/pom.xml                                     |   6 +
 43 files changed, 1239 insertions(+), 574 deletions(-)

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 8601ffd..03536e2 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
@@ -68,9 +68,9 @@ import static org.mockito.Mockito.verify;
  * Meta storage client tests.
  */
 @SuppressWarnings("WeakerAccess")
-public class MetaStorageServiceTest {
+public class ITMetaStorageServiceTest {
     /** The logger. */
-    private static final IgniteLogger LOG = IgniteLogger.forClass(MetaStorageServiceTest.class);
+    private static final IgniteLogger LOG = IgniteLogger.forClass(ITMetaStorageServiceTest.class);
 
     /** Base network port. */
     private static final int NODE_PORT_BASE = 20_000;
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 dc76a7c..fb71816 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,6 +20,7 @@ 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;
@@ -46,14 +47,17 @@ public class CursorImpl<T> implements Cursor<T> {
 
     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.it = new InnerIterator<>();
+        this.fn = fn;
     }
 
     /** {@inheritDoc} */
@@ -84,7 +88,7 @@ public class CursorImpl<T> implements Cursor<T> {
         return it.next();
     }
 
-    private class InnerIterator implements Iterator<T> {
+    private class InnerIterator<T> implements Iterator<T> {
         /** {@inheritDoc} */
         @Override public boolean hasNext() {
             try {
@@ -101,8 +105,8 @@ public class CursorImpl<T> implements Cursor<T> {
         /** {@inheritDoc} */
         @Override public T next() {
             try {
-                return initOp.thenCompose(
-                        cursorId -> metaStorageRaftGrpSvc.<T>run(new CursorNextCommand(cursorId))).get();
+                return (T)initOp.thenCompose(
+                        cursorId -> metaStorageRaftGrpSvc.run(new CursorNextCommand(cursorId))).thenApply(fn).get();
             }
             catch (InterruptedException | ExecutionException e) {
                 LOG.error("Unable to evaluate cursor hasNext command", 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 58%
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 4dd6bad..66a5328 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,63 +15,79 @@
  * 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.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 Serializable {
+public final class EntryImpl implements Entry {
     /** Key. */
-    @NotNull private ByteArray key;
+    private final ByteArray key;
 
     /** Value. */
-    @Nullable private byte[] val;
+    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 ByteArray key, @Nullable byte[] val, long revision, long updateCntr) {
+    EntryImpl(ByteArray key, 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} */
-    public @NotNull ByteArray key() {
+    /**
+     * Returns key.
+     *
+     * @return Key.
+     */
+    @Override public @NotNull ByteArray key() {
         return key;
     }
 
-    /** {@inheritDoc} */
-    public @Nullable byte[] value() {
+    /**
+     * Returns value.
+     *
+     * @return Value.
+     */
+    @Override public @Nullable byte[] value() {
         return val;
     }
 
-    /** {@inheritDoc} */
-    public long revision() {
-        return revision;
+    /**
+     * Returns revision.
+     *
+     * @return Revision.
+     */
+    @Override public long revision() {
+        return rev;
     }
 
-    /** {@inheritDoc} */
-    public long updateCounter() {
-        return updateCntr;
+    /**
+     * Returns update counter.
+     *
+     * @return Update counter.
+     */
+    @Override public long updateCounter() {
+        return updCntr;
     }
 
     /** {@inheritDoc} */
@@ -82,12 +98,12 @@ public final class DummyEntry implements Serializable {
         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))
@@ -99,9 +115,13 @@ public final class DummyEntry implements Serializable {
     /** {@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 fbd1c17..2764acc 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,24 +19,30 @@ 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;
@@ -44,12 +50,7 @@ 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.MetaStorageService;
-import org.apache.ignite.metastorage.client.Condition;
-import org.apache.ignite.metastorage.client.Entry;
-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.metastorage.client.*;
 import org.apache.ignite.raft.client.service.RaftGroupService;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -78,24 +79,29 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key) {
-        return metaStorageRaftGrpSvc.run(new GetCommand(key));
+        return metaStorageRaftGrpSvc.run(new GetCommand(key)).thenApply(obj -> {
+            SingleEntryResponse resp = (SingleEntryResponse)obj;
+
+            return new EntryImpl(new ByteArray(resp.key()), resp.value(), resp.revision(), resp.updateCounter());
+        });
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Entry> get(@NotNull ByteArray key, long revUpperBound) {
-        return metaStorageRaftGrpSvc.run(new GetCommand(key, revUpperBound));
+        return metaStorageRaftGrpSvc.run(new GetCommand(key, revUpperBound))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys) {
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+        return metaStorageRaftGrpSvc.run(new GetAllCommand(keys))
+                .thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAll(Collection<ByteArray> keys, long revUpperBound) {
-        return metaStorageRaftGrpSvc.<Collection<Entry>>run(new GetAllCommand(keys, revUpperBound)).
-            thenApply(entries -> entries.stream().collect(Collectors.toMap(Entry::key, Function.identity())));
+        return metaStorageRaftGrpSvc.run(new GetAllCommand(keys, revUpperBound)).
+                thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
@@ -105,7 +111,8 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Entry> getAndPut(@NotNull ByteArray key, @NotNull byte[] value) {
-        return metaStorageRaftGrpSvc.run(new GetAndPutCommand(key, value));
+        return metaStorageRaftGrpSvc.run(new GetAndPutCommand(key, value))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
@@ -115,16 +122,8 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndPutAll(@NotNull Map<ByteArray, byte[]> vals) {
-        List<ByteArray> 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())));
+        return metaStorageRaftGrpSvc.run(new GetAndPutAllCommand(vals)).
+            thenApply(MetaStorageServiceImpl::multipleEntryResult);
     }
 
     /** {@inheritDoc} */
@@ -134,46 +133,59 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Entry> getAndRemove(@NotNull ByteArray key) {
-        return metaStorageRaftGrpSvc.run(new GetAndRemoveCommand(key));
+        return metaStorageRaftGrpSvc.run(new GetAndRemoveCommand(key))
+                .thenApply(MetaStorageServiceImpl::singleEntryResult);
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys) {
+    @Override public @NotNull CompletableFuture<Void> removeAll(@NotNull Set<ByteArray> keys) {
         return metaStorageRaftGrpSvc.run(new RemoveAllCommand(keys));
     }
 
     /** {@inheritDoc} */
-    @Override public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Collection<ByteArray> 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: implement
-    @Override
-    public @NotNull CompletableFuture<Boolean> invoke(@NotNull Condition condition, @NotNull Operation success, @NotNull 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 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 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 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 ByteArray keyFrom, @Nullable ByteArray keyTo) {
         return new CursorImpl<>(
-            metaStorageRaftGrpSvc,
-            metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo))
+                metaStorageRaftGrpSvc,
+                metaStorageRaftGrpSvc.run(new RangeCommand(keyFrom, keyTo)),
+                MetaStorageServiceImpl::singleEntryResult
         );
     }
 
@@ -190,7 +202,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         watchRes.thenAccept(
             watchId -> watchProcessor.addWatch(
                 watchId,
-                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes),
+                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
                 lsnr
             )
         );
@@ -219,7 +231,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
         watchRes.thenAccept(
             watchId -> watchProcessor.addWatch(
                 watchId,
-                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes),
+                new CursorImpl<>(metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
                 lsnr
             )
         );
@@ -232,12 +244,105 @@ 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);
+            }
+
+            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);
+        }
+
+        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 = null;
+
+        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 {
@@ -308,8 +413,6 @@ 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;
-
                 Iterator<WatchEvent> watchEvtsIter = cursor.iterator();
 
                 while (true) {
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
index e060b8d..42f87cb 100644
--- 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
@@ -18,6 +18,7 @@
 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.
@@ -38,19 +39,20 @@ public final class Condition {
         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 type of condition.
-         *
-         * @see Type
-         */
-        private Type type;
-
         /** The revision as the condition argument. */
         private long rev;
 
@@ -63,130 +65,116 @@ public final class Condition {
             super(key);
         }
 
+        public long revision() {
+            return rev;
+        }
+
         /**
-         * Produces the condition of type {@link Type#EQUAL}. This condition tests the given revision on equality with
+         * 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 Type#EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.REV_EQUAL);
 
-            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
+         * 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 Type#NOT_EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.REV_NOT_EQUAL);
 
-            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
+         * 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 Type#GREATER}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.REV_GREATER);
 
-            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
+         * 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 Type#GREATER_OR_EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.REV_GREATER_OR_EQUAL);
 
-            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
+         * 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 Type#LESS}.
+         * @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);
+            validate(type());
 
-            this.type = Type.LESS;
+            type(ConditionType.REV_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
+         * 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 Type#LESS_OR_EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.REV_LESS_OR_EQUAL);
 
-            this.type = Type.LESS_OR_EQUAL;
             this.rev = rev;
 
             return new Condition(this);
         }
-
-        /**
-         * Defines possible condition types which can be applied to the revision.
-         */
-        enum Type {
-            /** Equality condition type. */
-            EQUAL,
-
-            /** Inequality condition type. */
-            NOT_EQUAL,
-
-            /** Greater than condition type. */
-            GREATER,
-
-            /** Less than condition type. */
-            LESS,
-
-            /** Less than or equal to condition type. */
-            LESS_OR_EQUAL,
-
-            /** Greater than or equal to condition type. */
-            GREATER_OR_EQUAL
-        }
     }
 
     /**
@@ -195,13 +183,6 @@ public final class Condition {
      * {@link IllegalStateException}.
      */
     public static final class ValueCondition extends AbstractCondition {
-        /**
-         * The type of condition.
-         *
-         * @see Type
-         */
-        private Type type;
-
         /** The value as the condition argument. */
         private byte[] val;
 
@@ -214,50 +195,45 @@ public final class Condition {
             super(key);
         }
 
+        public byte[] value() {
+            return val;
+        }
+
         /**
-         * Produces the condition of type {@link Type#EQUAL}. This condition tests the given value on equality with
+         * 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 Type#EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.VAL_EQUAL);
 
-            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
+         * 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 Type#NOT_EQUAL}.
+         * @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);
+            validate(type());
+
+            type(ConditionType.VAL_NOT_EQUAL);
 
-            this.type = Type.NOT_EQUAL;
             this.val = val;
 
             return new Condition(this);
         }
-
-        /**
-         * Defines possible condition types which can be applied to the value.
-         */
-        enum Type {
-            /** Equality condition type. */
-            EQUAL,
-
-            /** Inequality condition type. */
-            NOT_EQUAL
-        }
     }
 
     /**
@@ -267,13 +243,6 @@ public final class Condition {
      */
     public static final class ExistenceCondition extends AbstractCondition {
         /**
-         * The type of condition.
-         *
-         * @see Type
-         */
-        private Type type;
-
-        /**
          * Constructs a condition on existence an entry identified by the given key.
          *
          * @param key Identifies an entry which condition will be applied to.
@@ -283,45 +252,34 @@ public final class Condition {
         }
 
         /**
-         * Produces the condition of type {@link Type#EXISTS}. This condition tests the existence of an entry
+         * 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 Type#EXISTS}.
+         * @return The condition of type {@link ConditionType#KEY_EXISTS}.
          * @throws IllegalStateException In case when the condition is already defined.
          */
         public Condition exists() {
-            validate(type);
+            validate(type());
 
-            this.type = Type.EXISTS;
+            type(ConditionType.KEY_EXISTS);
 
             return new Condition(this);
         }
 
         /**
-         * Produces the condition of type {@link Type#NOT_EXISTS}. This condition tests the non-existence of an entry
+         * 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 Type#NOT_EXISTS}.
+         * @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);
+            validate(type());
 
-            this.type = Type.NOT_EXISTS;
+            type(ConditionType.KEY_NOT_EXISTS);
 
             return new Condition(this);
         }
-
-        /**
-         * Defines possible condition types which can be applied to the value.
-         */
-        enum Type {
-            /** Existence condition type. */
-            EXISTS,
-
-            /** Non-existence condition type. */
-            NOT_EXISTS
-        }
     }
 
     /**
@@ -344,6 +302,8 @@ public final class Condition {
          * @return Key which identifies an entry which condition will be applied to.
          */
         byte[] key();
+
+        ConditionType type();
     }
 
     /**
@@ -354,6 +314,11 @@ public final class Condition {
         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.
@@ -370,5 +335,13 @@ public final class Condition {
         @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-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 09757c3..35a7af8 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,6 +19,7 @@ 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;
@@ -167,14 +168,14 @@ public interface MetaStorageService {
     /**
      * 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 ByteArray
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys);
+    CompletableFuture<Void> removeAll(@NotNull Set<ByteArray> keys);
 
     /**
      * Removes entries for given keys and retrieves previous entries.
@@ -188,8 +189,7 @@ public interface MetaStorageService {
      * @see Entry
      */
     @NotNull
-    CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Collection<ByteArray> keys);
-
+    CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Set<ByteArray> keys);
 
     /**
      * Updates an entry for the given key conditionally.
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
index bd41824..d81d488 100644
--- 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
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.metastorage.client;
 
+import org.apache.ignite.internal.metastorage.common.OperationType;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -36,6 +38,20 @@ public final class Operation {
         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>.
      */
@@ -46,7 +62,7 @@ public final class Operation {
          * @param key Identifies an entry which operation will be applied to.
          */
         RemoveOp(byte[] key) {
-            super(key);
+            super(key, OperationType.REMOVE);
         }
     }
 
@@ -64,10 +80,19 @@ public final class Operation {
          * @param val The value to which the entry should be updated.
          */
         PutOp(byte[] key, byte[] val) {
-            super(key);
+            super(key, OperationType.PUT);
 
             this.val = val;
         }
+
+        /**
+         * Returns value.
+         *
+         * @return Value.
+         */
+        public byte[] value() {
+            return val;
+        }
     }
 
     /**
@@ -78,7 +103,7 @@ public final class Operation {
          * Default no-op constructor.
          */
         NoOp() {
-            super(null);
+            super(null, OperationType.NO_OP);
         }
     }
 
@@ -86,19 +111,57 @@ public final class Operation {
      * 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;
 
-        public AbstractOp(@Nullable byte[] key) {
+        /** Operation type. */
+        @NotNull private final OperationType type;
+
+        /**
+         * Ctor.
+         * @param key Key.
+         * @param type Operation type.
+         */
+        public 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/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/internal/metastorage/common/command/RemoveCommand.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/OperationType.java
similarity index 53%
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/OperationType.java
index 18322b9..7c97859 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/OperationType.java
@@ -15,30 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.metastorage.common.command;
-
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
+package org.apache.ignite.internal.metastorage.common;
 
 /**
- * Remove command for MetaStorageCommandListener that removes an entry for the given key.
+ * Defines possible operation types.
  */
-public final class RemoveCommand implements WriteCommand {
-    /** The key. Couldn't be {@code null}. */
-    @NotNull private final ByteArray key;
+public enum OperationType {
+    /** No-op operation. */
+    NO_OP,
 
-    /**
-     * @param key he key. Couldn't be {@code null}.
-     */
-    public RemoveCommand(@NotNull ByteArray key) {
-        this.key = key;
-    }
+    /** Put (insert/replace) operation. */
+    PUT,
 
-    /**
-     * @return The key. Couldn't be {@code null}.
-     */
-    public @NotNull ByteArray key() {
-        return key;
-    }
+    /** Remove operation. */
+    REMOVE
 }
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/ConditionInfo.java
similarity index 50%
copy from modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/GetCommand.java
copy to modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/ConditionInfo.java
index c71479b..998be42 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/ConditionInfo.java
@@ -17,51 +17,73 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.raft.client.ReadCommand;
-import org.jetbrains.annotations.NotNull;
+import java.io.Serializable;
+import org.apache.ignite.internal.metastorage.common.ConditionType;
 
 /**
- * Get command for MetaStorageCommandListener that retrieves an entry
- * for the given key and the revision upper bound, if latter is present.
+ * Defines condition for InvokeCommand.
  */
-public final class GetCommand implements ReadCommand {
+public class ConditionInfo implements Serializable {
     /** Key. */
-    @NotNull private final ByteArray key;
+    private final byte[] key;
 
-    /** The upper bound for entry revisions. Must be positive. */
-    private long revUpperBound;
+    /** Type. */
+    private final ConditionType type;
+
+    /** Value. */
+    private final byte[] val;
+
+    /** Revision. */
+    private final long rev;
 
     /**
-     * @param key Key. Couldn't be {@code null}.
+     * Construct condition with given parameters.
+     *
+     * @param key Key.
+     * @param type Condition type.
+     * @param val Value.
+     * @param rev Revision.
      */
-    public GetCommand(@NotNull ByteArray key) {
+    public ConditionInfo(byte[] key, ConditionType type, byte[] val, long rev) {
         this.key = key;
+        this.type = type;
+        this.val = val;
+        this.rev = rev;
     }
 
     /**
-     * @param key Key. Couldn't be {@code null}.
-     * @param revUpperBound The upper bound for entry revisions. Must be positive.
+     * Returns key.
+     *
+     * @return Key.
      */
-    public GetCommand(@NotNull ByteArray key, @NotNull long revUpperBound) {
-        this.key = key;
-
-        assert revUpperBound > 0;
+    public byte[] key() {
+        return key;
+    }
 
-        this.revUpperBound = revUpperBound;
+    /**
+     * Returns condition type.
+     *
+     * @return Condition type.
+     */
+    public ConditionType type() {
+        return type;
     }
 
     /**
-     * @return Key.
+     * Returns value.
+     *
+     * @return Value.
      */
-    public @NotNull ByteArray key() {
-        return key;
+    public byte[] value() {
+        return val;
     }
 
     /**
-     * @return The upper bound for entry revisions, or {@code null} if wasn't specified.
+     * Returns revision.
+     *
+     * @return Revision.
      */
     public long revision() {
-        return revUpperBound;
+        return rev;
     }
 }
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/ErrorResponse.java
similarity index 59%
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/ErrorResponse.java
index 18322b9..8ef1744 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/ErrorResponse.java
@@ -17,28 +17,28 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
+import java.io.Serializable;
 
-/**
- * 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 ByteArray key;
+/** Defines error response. */
+public class ErrorResponse implements Serializable {
+    /** Error code. */
+    private final int errCode;
 
     /**
-     * @param key he key. Couldn't be {@code null}.
+     * Constructs error response.
+     *
+     * @param errCode Error code
      */
-    public RemoveCommand(@NotNull ByteArray key) {
-        this.key = key;
+    public ErrorResponse(int errCode) {
+        this.errCode = errCode;
     }
 
     /**
-     * @return The key. Couldn't be {@code null}.
+     * Returns error code.
+     *
+     * @return Error code.
      */
-    public @NotNull ByteArray key() {
-        return key;
+    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 c5fcaa9..e20a793 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,24 +17,23 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 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 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<ByteArray> 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
@@ -43,10 +42,10 @@ public final class GetAllCommand implements ReadCommand {
     public GetAllCommand(@NotNull Collection<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());
     }
 
     /**
@@ -54,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<ByteArray> keys, @NotNull Long revUpperBound) {
+    public GetAllCommand(@NotNull Collection<ByteArray> keys, long revUpperBound) {
         this(keys);
 
         assert revUpperBound > 0;
@@ -63,16 +62,16 @@ public final class GetAllCommand implements ReadCommand {
     }
 
     /**
-     * @return The collection of keys.
+     * @return The list of keys.
      */
-    public @NotNull Collection<ByteArray> 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 d586624..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,8 +17,9 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
@@ -29,27 +30,31 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class GetAndPutAllCommand implements WriteCommand {
     /** Keys. */
-    @NotNull private final List<ByteArray> keys;
+    @NotNull private final List<byte[]> keys;
 
     /** Values. */
     @NotNull private final List<byte[]> vals;
 
     /**
-     * @param keys Keys.
      * @param vals Values.
      */
-    public GetAndPutAllCommand(@NotNull List<ByteArray> 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<ByteArray> 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 9bcb8aa..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
@@ -27,7 +27,7 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class GetAndPutCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final ByteArray key;
+    @NotNull private final byte[] key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,14 +37,14 @@ public final class GetAndPutCommand implements WriteCommand {
      * @param val The value. Couldn't be {@code null}.
      */
     public GetAndPutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
-        this.key = key;
+        this.key = key.bytes();
         this.val = val;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull ByteArray 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 5712f0b..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,9 +17,9 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
+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;
@@ -30,24 +30,22 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class GetAndRemoveAllCommand implements WriteCommand {
     /** The keys collection. Couldn't be {@code null}. */
-    @NotNull private final Collection<ByteArray> keys;
+    @NotNull private final List<byte[]> keys;
 
     /**
      * @param keys The keys collection. Couldn't be {@code null}.
      */
-    public GetAndRemoveAllCommand(@NotNull Collection<ByteArray> 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<ByteArray> 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 b4a4166..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
@@ -27,19 +27,19 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class GetAndRemoveCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final ByteArray key;
+    @NotNull private final byte[] key;
 
     /**
      * @param key The key. Couldn't be {@code null}.
      */
     public GetAndRemoveCommand(@NotNull ByteArray key) {
-        this.key = key;
+        this.key = key.bytes();
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull ByteArray 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 c71479b..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
@@ -27,7 +27,7 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class GetCommand implements ReadCommand {
     /** Key. */
-    @NotNull private final ByteArray key;
+    @NotNull private final byte[] key;
 
     /** The upper bound for entry revisions. Must be positive. */
     private long revUpperBound;
@@ -36,15 +36,15 @@ public final class GetCommand implements ReadCommand {
      * @param key Key. Couldn't be {@code null}.
      */
     public GetCommand(@NotNull ByteArray key) {
-        this.key = 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 ByteArray key, @NotNull long revUpperBound) {
-        this.key = key;
+    public GetCommand(@NotNull ByteArray key, long revUpperBound) {
+        this.key = key.bytes();
 
         assert revUpperBound > 0;
 
@@ -54,7 +54,7 @@ public final class GetCommand implements ReadCommand {
     /**
      * @return Key.
      */
-    public @NotNull ByteArray key() {
+    public @NotNull byte[] key() {
         return key;
     }
 
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..6981a0e
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/command/InvokeCommand.java
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.ignite.raft.client.Command;
+
+import java.util.List;
+
+/**
+ * Represents invoke command for meta storage.
+ */
+public class InvokeCommand implements Command {
+    /** 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 59%
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 18322b9..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,28 +17,31 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
+import java.io.Serializable;
+import java.util.List;
 
 /**
- * Remove command for 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 ByteArray 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 ByteArray 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 ByteArray 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 53%
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 1efff24..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,41 +17,60 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import org.apache.ignite.lang.ByteArray;
-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 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 ByteArray 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 ByteArray 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 ByteArray 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 029ba99..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,8 +17,8 @@
 
 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.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
@@ -29,8 +29,11 @@ import org.jetbrains.annotations.NotNull;
  * 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<ByteArray, 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.
@@ -38,16 +41,37 @@ public final class PutAllCommand implements WriteCommand {
     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<ByteArray, 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 1efff24..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
@@ -27,7 +27,7 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class PutCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final ByteArray key;
+    @NotNull private final byte[] key;
 
     /** The value. Couldn't be {@code null}. */
     @NotNull private final byte[] val;
@@ -37,14 +37,14 @@ public final class PutCommand implements WriteCommand {
      * @param val The value. Couldn't be {@code null}.
      */
     public PutCommand(@NotNull ByteArray key, @NotNull byte[] val) {
-        this.key = key;
+        this.key = key.bytes();
         this.val = val;
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull ByteArray 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 8053f69..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
@@ -28,10 +28,10 @@ import org.jetbrains.annotations.Nullable;
  */
 public final class RangeCommand implements WriteCommand {
     /** Start key of range (inclusive). Couldn't be {@code null}. */
-    @NotNull private final ByteArray keyFrom;
+    @NotNull private final byte[] keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final ByteArray keyTo;
+    @Nullable private final byte[] keyTo;
 
     /** The upper bound for entry revision. {@code -1} means latest revision. */
     @NotNull private final long revUpperBound;
@@ -52,24 +52,24 @@ public final class RangeCommand implements WriteCommand {
     public RangeCommand(
         @NotNull ByteArray keyFrom,
         @Nullable ByteArray keyTo,
-        @NotNull long revUpperBound
+        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 ByteArray keyFrom() {
+    public @NotNull byte[] keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable ByteArray keyTo() {
+    public @Nullable byte[] keyTo() {
         return keyTo;
     }
 
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 4edad35..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,9 +17,9 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
+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;
@@ -28,23 +28,23 @@ import org.jetbrains.annotations.NotNull;
  * 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<ByteArray> 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<ByteArray> 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<ByteArray> 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 18322b9..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
@@ -26,19 +26,19 @@ import org.jetbrains.annotations.NotNull;
  */
 public final class RemoveCommand implements WriteCommand {
     /** The key. Couldn't be {@code null}. */
-    @NotNull private final ByteArray key;
+    @NotNull private final byte[] key;
 
     /**
      * @param key he key. Couldn't be {@code null}.
      */
     public RemoveCommand(@NotNull ByteArray key) {
-        this.key = key;
+        this.key = key.bytes();
     }
 
     /**
      * @return The key. Couldn't be {@code null}.
      */
-    public @NotNull ByteArray 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 9a0c628..efde780 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,9 +17,9 @@
 
 package org.apache.ignite.internal.metastorage.common.command;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
 import org.jetbrains.annotations.NotNull;
@@ -28,29 +28,29 @@ import org.jetbrains.annotations.NotNull;
  * 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<ByteArray> 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<ByteArray> keys, @NotNull Long revision) {
-        if (keys instanceof Serializable)
-            this.keys = keys;
-        else
-            this.keys = new ArrayList<>(keys);
+    public WatchExactKeysCommand(@NotNull Collection<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<ByteArray> 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 f164642..80e4b46 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
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.metastorage.common.command;
 
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.raft.client.WriteCommand;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -27,10 +26,10 @@ import org.jetbrains.annotations.Nullable;
  */
 public final class WatchRangeKeysCommand implements WriteCommand {
     /** Start key of range (inclusive). Couldn't be {@code null}. */
-    @Nullable private final ByteArray keyFrom;
+    @Nullable private final byte[] keyFrom;
 
     /** End key of range (exclusive). Could be {@code null}. */
-    @Nullable private final ByteArray keyTo;
+    @Nullable private final byte[] keyTo;
 
     /** Start revision inclusive. {@code 0} - all revisions. */
     private final long revision;
@@ -51,24 +50,24 @@ public final class WatchRangeKeysCommand implements WriteCommand {
     public WatchRangeKeysCommand(
         @Nullable ByteArray keyFrom,
         @Nullable ByteArray keyTo,
-        @NotNull long revision
+        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 ByteArray keyFrom() {
+    public @Nullable byte[] keyFrom() {
         return keyFrom;
     }
 
     /**
      * @return End key of range (exclusive). Could be {@code null}.
      */
-    public @Nullable ByteArray keyTo() {
+    public @Nullable byte[] keyTo() {
         return keyTo;
     }
 
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
similarity index 50%
copy from modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
copy to modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
index c1e4f56..85cbf3d 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/CompactedException.java
@@ -18,41 +18,42 @@
 package org.apache.ignite.internal.metastorage.server;
 
 /**
- * Represent an update event for particular key and entry.
+ * Thrown when a requested operation on meta storage could not be performed because target revisions were removed
+ * from storage due to a compaction procedure. In such case the operation should be retried with actual revision.
  */
-public class EntryEvent {
-    /** Old (previous) entry. */
-    private final Entry oldEntry;
-
-    /** New (current) entry. */
-    private final Entry entry;
+public class CompactedException extends RuntimeException {
+    /**
+     * Constructs an exception.
+     */
+    public CompactedException() {
+        super();
+    }
 
     /**
-     * Constructs event with given old and new entries.
+     * Constructs an exception with a given message.
      *
-     * @param oldEntry Old entry.
-     * @param curEntry New entry.
+     * @param message Detail message.
      */
-    EntryEvent(Entry oldEntry, Entry curEntry) {
-        this.oldEntry = oldEntry;
-        this.entry = curEntry;
+    public CompactedException(String message) {
+        super(message);
     }
 
     /**
-     * Returns old entry.
+     * Constructs an exception with a given message and a cause.
      *
-     * @return Old entry.
+     * @param message Detail message.
+     * @param cause Cause.
      */
-    public Entry oldEntry() {
-        return oldEntry;
+    public CompactedException(String message, Throwable cause) {
+        super(message, cause);
     }
 
     /**
-     * Rreturns new entry.
+     * Constructs an exception with a given cause.
      *
-     * @return New entry.
+     * @param cause Cause.
      */
-    public Entry entry() {
-        return entry;
+    public CompactedException(Throwable cause) {
+        super(cause);
     }
 }
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
index 87b5471..22f1aec 100644
--- 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
@@ -36,7 +36,6 @@ import org.jetbrains.annotations.Nullable;
  *     <li>A tombstone entry which denotes that a regular entry for a given key was removed from storage on some revision.</li>
  * </ul>
  */
-//TODO: Separate client and server entries. Empty and tombstone for client is the same.
 public class Entry {
     /** Entry key. Couldn't be {@code null}. */
     @NotNull
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
index c1e4f56..ef0adba 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/EntryEvent.java
@@ -33,7 +33,7 @@ public class EntryEvent {
      * @param oldEntry Old entry.
      * @param curEntry New entry.
      */
-    EntryEvent(Entry oldEntry, Entry curEntry) {
+    public EntryEvent(Entry oldEntry, Entry curEntry) {
         this.oldEntry = oldEntry;
         this.entry = curEntry;
     }
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
index f46610c..d4c7da5 100644
--- 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
@@ -22,54 +22,185 @@ 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();
 
-    @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);
-
+    /**
+     * 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);
 
-    @NotNull
-    Collection<Entry> getAndPutAll(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);
 
-    @NotNull
-    Entry getAndRemove(byte[] key);
-
-    void removeAll(List<byte[]> key);
-
-    @NotNull
-    Collection<Entry> getAndRemoveAll(List<byte[]> keys);
-
+    /**
+     * 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
index aaea75b..4c69d7d 100644
--- 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
@@ -18,6 +18,7 @@
 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;
 
@@ -26,19 +27,19 @@ import org.jetbrains.annotations.Nullable;
  * <p>
  * Invariants:
  * <ul>
- *     <li>Any operation identifies a target entry by not null {@code key} except of {@link Type#NO_OP}.</li>
- *     <li>Only {@link Type#PUT} operation contains value which will be written to meta storage.</li>
+ *     <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>
  */
-final class Operation {
+public final class Operation {
     /**
-     * Key identifies an entry which operation will be applied to. Key is {@code null} for {@link Type#NO_OP} 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 Type#PUT}
+     * Value which will be associated with the {@link #key}. Value is not {@code null} only for {@link OperationType#PUT}
      * operation.
      */
     @Nullable
@@ -46,10 +47,10 @@ final class Operation {
 
     /**
      * Operation type.
-     * @see Type
+     * @see OperationType
      */
     @NotNull
-    private final Type type;
+    private final OperationType type;
 
     /**
      * Constructs operation which will be applied to an entry identified by the given key.
@@ -58,10 +59,10 @@ final class Operation {
      * @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}.
      */
-    Operation(@NotNull Type type, @Nullable byte[] key, @Nullable byte[] val) {
-        assert (type == Type.NO_OP && key == null && val == null)
-                || (type == Type.PUT && key != null && val != null)
-                || (type == Type.REMOVE && key != null && val == null)
+    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") + ']';
 
@@ -75,7 +76,7 @@ final class Operation {
      *
      * @return A key which identifies an entry which operation will be applied to.
      */
-    @Nullable byte[] key() {
+    @Nullable public byte[] key() {
         return key;
     }
 
@@ -84,7 +85,7 @@ final class Operation {
      *
      * @return A value which will be associated with an entry identified by the {@code key}.
      */
-    @Nullable byte[] value() {
+    @Nullable public byte[] value() {
         return val;
     }
 
@@ -93,19 +94,7 @@ final class Operation {
      *
      * @return An operation type.
      */
-    @NotNull Type type() {
+    @NotNull public OperationType type() {
         return type;
     }
-
-    /** Defines operation types. */
-    enum Type {
-        /** Put operation. */
-        PUT,
-
-        /** Remove operation. */
-        REMOVE,
-
-        /** No-op operation. */
-        NO_OP
-    }
 }
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
index 1f3f0e4..d37f0bb 100644
--- 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
@@ -472,8 +472,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                 // Update keysIdx.
                 List<Long> revs = keysIdx.computeIfAbsent(key, k -> new ArrayList<>());
 
-                long lastRev = revs.isEmpty() ? 0 : lastRevision(revs);
-
                 revs.add(curRev);
 
                 Value val = new Value(bytes, curUpdCntr);
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
index 444bfdb..f33ff9b 100644
--- 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
@@ -56,7 +56,7 @@ public class ValueCondition extends AbstractCondition {
     /**
      * Defines possible condition types which can be applied to the value.
      */
-    enum Type {
+    public enum Type {
         /** Equality condition type. */
         EQUAL {
             @Override public boolean test(long res) {
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
index 71b4f34..df6966f 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageCommandListener.java
@@ -17,35 +17,45 @@
 
 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.ByteArray;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteLogger;
 import org.apache.ignite.lang.IgniteUuid;
@@ -53,7 +63,6 @@ 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.
@@ -66,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.
@@ -85,32 +94,42 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
                 if (clo.command() instanceof GetCommand) {
                     GetCommand getCmd = (GetCommand)clo.command();
 
+                    Entry e;
+
                     if (getCmd.revision() != 0)
-                        clo.success(storage.get(getCmd.key().bytes(), getCmd.revision()));
+                        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(ByteArray::bytes).collect(Collectors.toList()),
-                            getAllCmd.revision())
-                        );
-                    }
-                    else {
-                        clo.success(storage.getAll(
-                            getAllCmd.keys().stream().map(ByteArray::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() + ']';
@@ -138,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(ByteArray::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(ByteArray::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(ByteArray::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(ByteArray::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();
@@ -206,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);
                 }
@@ -223,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();
@@ -248,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);
                 }
@@ -265,14 +312,9 @@ public class MetaStorageCommandListener implements RaftGroupCommandListener {
 
                     IgniteUuid cursorId = new IgniteUuid(UUID.randomUUID(), 0L);
 
-                    Cursor<WatchEvent> cursor = storage.watch(
-                        watchCmd.keys().stream().map(ByteArray::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);
                 }
@@ -292,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();
+    }
+
+    /** */
+    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/SimpleInMemoryKeyValueStorageTest.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
index 8e138fd..9fe8ba7 100644
--- 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
@@ -23,11 +23,11 @@ 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;
@@ -37,6 +37,9 @@ 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;
 
@@ -967,10 +970,10 @@ class SimpleInMemoryKeyValueStorageTest {
         boolean branch = storage.invoke(
                 new RevisionCondition(RevisionCondition.Type.EQUAL, key1, 1),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 ),
-                List.of(new Operation(Operation.Type.PUT, key3, val3))
+                List.of(new Operation(OperationType.PUT, key3, val3))
         );
 
         // "Success" branch is applied.
@@ -1022,10 +1025,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         boolean branch = storage.invoke(
                 new RevisionCondition(RevisionCondition.Type.EQUAL, key1, 2),
-                List.of(new Operation(Operation.Type.PUT, key3, val3)),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 )
         );
 
@@ -1079,10 +1082,10 @@ class SimpleInMemoryKeyValueStorageTest {
         boolean branch = storage.invoke(
                 new ExistenceCondition(ExistenceCondition.Type.EXISTS, key1),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 ),
-                List.of(new Operation(Operation.Type.PUT, key3, val3))
+                List.of(new Operation(OperationType.PUT, key3, val3))
         );
 
         // "Success" branch is applied.
@@ -1134,10 +1137,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         boolean branch = storage.invoke(
                 new ExistenceCondition(ExistenceCondition.Type.EXISTS, key3),
-                List.of(new Operation(Operation.Type.PUT, key3, val3)),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 )
         );
 
@@ -1191,10 +1194,10 @@ class SimpleInMemoryKeyValueStorageTest {
         boolean branch = storage.invoke(
                 new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key2),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 ),
-                List.of(new Operation(Operation.Type.PUT, key3, val3))
+                List.of(new Operation(OperationType.PUT, key3, val3))
         );
 
         // "Success" branch is applied.
@@ -1246,10 +1249,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         boolean branch = storage.invoke(
                 new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key1),
-                List.of(new Operation(Operation.Type.PUT, key3, val3)),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 )
         );
 
@@ -1303,10 +1306,10 @@ class SimpleInMemoryKeyValueStorageTest {
         boolean branch = storage.invoke(
                 new ValueCondition(ValueCondition.Type.EQUAL, key1, val1_1),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 ),
-                List.of(new Operation(Operation.Type.PUT, key3, val3))
+                List.of(new Operation(OperationType.PUT, key3, val3))
         );
 
         // "Success" branch is applied.
@@ -1358,10 +1361,10 @@ class SimpleInMemoryKeyValueStorageTest {
 
         boolean branch = storage.invoke(
                 new ValueCondition(ValueCondition.Type.EQUAL, key1, val1_2),
-                List.of(new Operation(Operation.Type.PUT, key3, val3)),
+                List.of(new Operation(OperationType.PUT, key3, val3)),
                 List.of(
-                        new Operation(Operation.Type.PUT, key1, val1_2),
-                        new Operation(Operation.Type.PUT, key2, val2)
+                        new Operation(OperationType.PUT, key1, val1_2),
+                        new Operation(OperationType.PUT, key2, val2)
                 )
         );
 
@@ -1414,8 +1417,8 @@ class SimpleInMemoryKeyValueStorageTest {
         // No-op.
         boolean branch = storage.invoke(
                 new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
-                List.of(new Operation(Operation.Type.NO_OP, null, null)),
-                List.of(new Operation(Operation.Type.NO_OP, null, null))
+                List.of(new Operation(OperationType.NO_OP, null, null)),
+                List.of(new Operation(OperationType.NO_OP, null, null))
         );
 
         assertTrue(branch);
@@ -1428,10 +1431,10 @@ class SimpleInMemoryKeyValueStorageTest {
         branch = storage.invoke(
                 new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
                 List.of(
-                        new Operation(Operation.Type.PUT, key2, val2),
-                        new Operation(Operation.Type.PUT, key3, val3)
+                        new Operation(OperationType.PUT, key2, val2),
+                        new Operation(OperationType.PUT, key3, val3)
                 ),
-                List.of(new Operation(Operation.Type.NO_OP, null, null))
+                List.of(new Operation(OperationType.NO_OP, null, null))
         );
 
         assertTrue(branch);
@@ -1462,10 +1465,10 @@ class SimpleInMemoryKeyValueStorageTest {
         branch = storage.invoke(
                 new ValueCondition(ValueCondition.Type.EQUAL, key1, val1),
                 List.of(
-                        new Operation(Operation.Type.REMOVE, key2, null),
-                        new Operation(Operation.Type.REMOVE, key3, null)
+                        new Operation(OperationType.REMOVE, key2, null),
+                        new Operation(OperationType.REMOVE, key3, null)
                 ),
-                List.of(new Operation(Operation.Type.NO_OP, null, null))
+                List.of(new Operation(OperationType.NO_OP, null, null))
         );
 
         assertTrue(branch);
diff --git a/modules/metastorage/pom.xml b/modules/metastorage/pom.xml
index 6c3a3f8..7e98589 100644
--- a/modules/metastorage/pom.xml
+++ b/modules/metastorage/pom.xml
@@ -55,12 +55,12 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>ignite-api</artifactId>
+            <artifactId>ignite-metastorage-server</artifactId>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>ignite-vault</artifactId>
+            <artifactId>ignite-api</artifactId>
         </dependency>
 
         <dependency>
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index f631db8..2249268 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Predicate;
@@ -29,6 +30,8 @@ import java.util.stream.Collectors;
 import org.apache.ignite.configuration.internal.ConfigurationManager;
 import org.apache.ignite.configuration.schemas.runner.NodeConfiguration;
 import org.apache.ignite.internal.metastorage.client.MetaStorageServiceImpl;
+import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageCommandListener;
 import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
 import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.util.Cursor;
@@ -128,19 +131,17 @@ public class MetaStorageManager {
 
         if (hasMetastorage(locNodeName, metastorageNodes)) {
 
-            //TODO:
-/*
             this.metaStorageSvcFut = CompletableFuture.completedFuture(new MetaStorageServiceImpl(
                     raftMgr.startRaftGroup(
                         METASTORAGE_RAFT_GROUP_NAME,
                         clusterNetSvc.topologyService().allMembers().stream().filter(
                             metaStorageNodesContainsLocPred).
                             collect(Collectors.toList()),
-                        new MetaStorageCommandListener(new KeyValueStorageImpl())
+                        new MetaStorageCommandListener(new SimpleInMemoryKeyValueStorage())
                     )
                 )
             );
-*/
+
         }
         else if (metastorageNodes.length > 0) {
             this.metaStorageSvcFut = CompletableFuture.completedFuture(new MetaStorageServiceImpl(
@@ -339,16 +340,16 @@ public class MetaStorageManager {
     }
 
     /**
-     * @see MetaStorageService#removeAll(Collection)
+     * @see MetaStorageService#removeAll(Set)
      */
-    public @NotNull CompletableFuture<Void> removeAll(@NotNull Collection<ByteArray> keys) {
+    public @NotNull CompletableFuture<Void> removeAll(@NotNull Set<ByteArray> keys) {
         return metaStorageSvcFut.thenCompose(svc -> svc.removeAll(keys));
     }
 
     /**
-     * @see MetaStorageService#getAndRemoveAll(Collection)
+     * @see MetaStorageService#getAndRemoveAll(Set)
      */
-    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Collection<ByteArray> keys) {
+    public @NotNull CompletableFuture<Map<ByteArray, Entry>> getAndRemoveAll(@NotNull Set<ByteArray> keys) {
         return metaStorageSvcFut.thenCompose(svc -> svc.getAndRemoveAll(keys));
     }
 
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
index b50d603..5482f87 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/WatchAggregatorTest.java
@@ -28,6 +28,8 @@ import org.apache.ignite.metastorage.client.WatchListener;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -49,19 +51,20 @@ public class WatchAggregatorTest {
                 entry("1", "value1n", 1, 1)
         );
 
-        var watchEvent1 = new WatchEvent(entryEvt1);
-
         var entryEvt2 = new EntryEvent(
                 entry("2", "value2", 1, 1),
                 entry("2", "value2n", 1, 1)
         );
 
-        var watchEvent2 = new WatchEvent(entryEvt2);
-
         watchAggregator.watch(1, (v1, v2) -> {}).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
 
-        verify(lsnr1).onUpdate(watchEvent1);
-        verify(lsnr2).onUpdate(watchEvent2);
+        var watchEvt1Res = ArgumentCaptor.forClass(WatchEvent.class);
+        verify(lsnr1).onUpdate(watchEvt1Res.capture());
+        assertEquals(List.of(entryEvt1), watchEvt1Res.getValue().entryEvents());
+
+        var watchEvt2Res = ArgumentCaptor.forClass(WatchEvent.class);
+        verify(lsnr2).onUpdate(watchEvt2Res.capture());
+        assertEquals(List.of(entryEvt2), watchEvt2Res.getValue().entryEvents());
     }
 
     @Test
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index 02ca745..c2710f8 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -202,7 +202,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                 UUID tblId = new UUID(revision, 0L);
 
                 if (hasMetastorageLocally) {
-                    var key = new ByteArray(INTERNAL_PREFIX + tblId.);
+                    var key = new ByteArray(INTERNAL_PREFIX + tblId);
 
                     futs.add(metaStorageMgr.invoke(
                         Conditions.exists(key),
diff --git a/parent/pom.xml b/parent/pom.xml
index 3f037e7..3cc852d 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -176,6 +176,12 @@
 
             <dependency>
                 <groupId>org.apache.ignite</groupId>
+                <artifactId>ignite-metastorage-server</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.ignite</groupId>
                 <artifactId>ignite-network</artifactId>
                 <version>${project.version}</version>
             </dependency>

[ignite-3] 05/09: IGNITE-14389 putAll initial (WIP)

Posted by ag...@apache.org.
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 54b1c420ae35d4ae831e53d4692c948caa0220ce
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Thu Apr 15 21:18:56 2021 +0300

    IGNITE-14389 putAll initial (WIP)
---
 .../apache/ignite/internal/metastorage/server/KeyValueStorage.java | 2 ++
 .../internal/metastorage/server/SimpleInMemoryKeyValueStorage.java | 7 +++++++
 2 files changed, 9 insertions(+)

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
index 0596c4a..0f18ece 100644
--- 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
@@ -29,6 +29,8 @@ public interface KeyValueStorage {
     @NotNull
     Entry getAndPut(byte[] key, byte[] value);
 
+    void putAll(List<byte[]> keys, List<byte[]> values);
+
     void remove(byte[] key);
 
     @NotNull
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
index 8523f51..f532005 100644
--- 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
@@ -63,6 +63,13 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         }
     }
 
+    @Override
+    public void putAll(List<byte[]> keys, List<byte[]> values) {
+        synchronized (mux) {
+
+        }
+    }
+
     @NotNull
     @Override public Entry get(byte[] key) {
         synchronized (mux) {

[ignite-3] 04/09: IGNITE-14389 getAll and tests (WIP)

Posted by ag...@apache.org.
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 7e0547d123de4afbec3bdfc7d6a22abc4dadfea2
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Tue Apr 13 21:34:37 2021 +0300

    IGNITE-14389 getAll and tests (WIP)
---
 .../metastorage/client/MetaStorageService.java     |   4 +-
 modules/metastorage-server/pom.xml                 |   6 +
 .../metastorage/server/KeyValueStorage.java        |   8 +
 .../server/SimpleInMemoryKeyValueStorage.java      |  36 +++-
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 208 +++++++++++++++++++--
 5 files changed, 244 insertions(+), 18 deletions(-)

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..2d0f9c9 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
@@ -52,7 +52,7 @@ public interface MetaStorageService {
      * Retrieves an entry for the given key and the revision upper bound.
      *
      * @param key The key. Couldn't be {@code null}.
-     * @param 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.
@@ -82,7 +82,7 @@ public interface MetaStorageService {
      *
      * @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 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.
diff --git a/modules/metastorage-server/pom.xml b/modules/metastorage-server/pom.xml
index 3c51fc5..d73d080 100644
--- a/modules/metastorage-server/pom.xml
+++ b/modules/metastorage-server/pom.xml
@@ -40,6 +40,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>metastorage-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>org.jetbrains</groupId>
             <artifactId>annotations</artifactId>
         </dependency>
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
index ead1043..0596c4a 100644
--- 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
@@ -2,7 +2,9 @@ package org.apache.ignite.internal.metastorage.server;
 
 import org.jetbrains.annotations.NotNull;
 
+import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 
 public interface KeyValueStorage {
 
@@ -16,6 +18,12 @@ public interface KeyValueStorage {
     @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
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
index 3700f4a..8523f51 100644
--- 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
@@ -2,6 +2,7 @@ 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;
@@ -78,6 +79,16 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     @Override
+    public @NotNull Collection<Entry> getAll(List<byte[]> keys) {
+        return doGetAll(keys, LATEST_REV);
+    }
+
+    @Override
+    public @NotNull Collection<Entry> getAll(List<byte[]> keys, long revUpperBound) {
+        return doGetAll(keys, revUpperBound);
+    }
+
+    @Override
     public void remove(byte[] key) {
         synchronized (mux) {
             Entry e = doGet(key, LATEST_REV, false);
@@ -197,17 +208,17 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
             NavigableMap<byte[], List<Long>> compactedKeysIdx,
             NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx
     ) {
-        Long lrev = lastRevision(revs);
+        Long lastRev = lastRevision(revs);
 
-        NavigableMap<byte[], Value> kv = revsIdx.get(lrev);
+        NavigableMap<byte[], Value> kv = revsIdx.get(lastRev);
 
         Value lastVal = kv.get(key);
 
         if (!lastVal.tombstone()) {
-            compactedKeysIdx.put(key, listOf(lrev));
+            compactedKeysIdx.put(key, listOf(lastRev));
 
             NavigableMap<byte[], Value> compactedKv = compactedRevsIdx.computeIfAbsent(
-                    lrev,
+                    lastRev,
                     k -> new TreeMap<>(LEXICOGRAPHIC_COMPARATOR)
             );
 
@@ -215,6 +226,23 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         }
     }
 
+    @NotNull
+    private Collection<Entry> doGetAll(List<byte[]> keys, long rev) {
+        assert keys != null : "keys list can't be null.";
+        assert !keys.isEmpty() : "keys list can't be empty.";
+        assert rev > 0 : "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;
+    }
+
     /**
      * Returns entry for given key.
      *
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
index 5b797fc..fa130e6 100644
--- 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
@@ -1,17 +1,18 @@
 package org.apache.ignite.internal.metastorage.server;
 
-import org.jetbrains.annotations.NotNull;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
+import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.ignite.metastorage.common.Key;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+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.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.*;
 
 class SimpleInMemoryKeyValueStorageTest {
     private KeyValueStorage storage;
@@ -56,6 +57,193 @@ class SimpleInMemoryKeyValueStorageTest {
     }
 
     @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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(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 Key(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<Key, Entry> map =  entries.stream().collect(Collectors.toMap(e -> new Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        Entry e1 = map.get(new Key(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 Key(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 Key(key3));
+
+        assertNotNull(e3);
+        assertTrue(e3.empty());
+
+        Entry e4 = map.get(new Key(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 Key(e.key()), Function.identity()));
+
+        // Test regular put value.
+        e1 = map.get(new Key(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 Key(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 Key(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 Key(key4));
+
+        assertNotNull(e4);
+        assertTrue(e4.empty());
+    }
+
+    @Test
     public void getAndPut() {
         byte[] key = k(1);
         byte[] val = kv(1, 1);
@@ -208,7 +396,6 @@ class SimpleInMemoryKeyValueStorageTest {
     @Test
     public void getAndPutAfterRemove() {
         byte[] key = k(1);
-
         byte[] val = kv(1, 1);
 
         storage.getAndPut(key, val);
@@ -218,11 +405,8 @@ class SimpleInMemoryKeyValueStorageTest {
         Entry e = storage.getAndPut(key, val);
 
         assertEquals(3, storage.revision());
-
         assertEquals(3, storage.updateCounter());
-
         assertEquals(2, e.revision());
-
         assertTrue(e.tombstone());
     }
 

[ignite-3] 02/09: IGNITE-14398: Meta storage: added update counter

Posted by ag...@apache.org.
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 de979bd01c314e87efcec46bffe7e296de4b19de
Author: Andrey Gura <ag...@apache.org>
AuthorDate: Wed Mar 31 18:18:09 2021 +0300

    IGNITE-14398: Meta storage: added update counter
---
 .../ignite/internal/metastorage/server/Entry.java  | 37 +++++++++---
 .../metastorage/server/KeyValueStorage.java        |  2 +
 .../server/SimpleInMemoryKeyValueStorage.java      | 70 +++++++++++++---------
 .../ignite/internal/metastorage/server/Value.java  | 27 +++++++++
 .../server/SimpleInMemoryKeyValueStorageTest.java  | 29 +++++++++
 5 files changed, 130 insertions(+), 35 deletions(-)

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
index 442aef9..263a88b 100644
--- 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
@@ -44,20 +44,31 @@ public class Entry {
     final private 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>
+     */
+    final private 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) {
+    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;
     }
 
     /**
@@ -65,13 +76,15 @@ public class Entry {
      *
      * @param key Key bytes. Couldn't be {@code null}.
      * @param rev Revision.
+     * @param updCntr Update counter.
      */
-    private Entry(@NotNull byte[] key, long rev) {
+    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;
     }
 
     /**
@@ -82,20 +95,22 @@ public class Entry {
      */
     @NotNull
     public static Entry empty(byte[] key) {
-        return new Entry(key, 0);
+        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) {
+    public static Entry tombstone(byte[] key, long rev, long updCntr) {
         assert rev > 0 : "rev must be positive for tombstone entry.";
 
-        return new Entry(key, rev);
+        return new Entry(key, rev, updCntr);
     }
 
     /**
@@ -127,12 +142,20 @@ public class Entry {
     }
 
     /**
+     * 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;
+        return val == null && rev > 0 && updCntr > 0;
     }
 
     /**
@@ -141,6 +164,6 @@ public class Entry {
      * @return {@code True} if entry is empty, otherwise - {@code false}.
      */
     public boolean empty() {
-        return val == null && rev == 0;
+        return val == null && rev == 0 && updCntr == 0;
     }
 }
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
index 1bf6b78..e245e08 100644
--- 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
@@ -8,6 +8,8 @@ public interface KeyValueStorage {
 
     long revision();
 
+    long updateCounter();
+
     @NotNull
     Entry put(byte[] key, byte[] value);
 
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
index 9059aec..b764998 100644
--- 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
@@ -12,23 +12,25 @@ import java.util.TreeMap;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.TestOnly;
 
+import static org.apache.ignite.internal.metastorage.server.Value.TOMBSTONE;
+
 /**
  * WARNING: Only for test purposes and only for non-distributed (one static instance) storage.
  */
 public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     private static final Comparator<byte[]> LEXICOGRAPHIC_COMPARATOR = Arrays::compare;
 
-    private static final byte[] TOMBSTONE = new byte[0];
-
     private static final long LATEST_REV = -1;
 
     private final Watcher watcher;
 
     private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
 
-    private NavigableMap<Long, NavigableMap<byte[], byte[]>> revsIdx = new TreeMap<>();
+    private NavigableMap<Long, NavigableMap<byte[], Value>> revsIdx = new TreeMap<>();
 
-    private long grev = 0;
+    private long rev;
+
+    private long updCntr;
 
     private final Object mux = new Object();
 
@@ -37,35 +39,45 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     @Override public long revision() {
-        return grev;
+        return rev;
+    }
+
+    @Override public long updateCounter() {
+        return updCntr;
     }
 
     @NotNull
-    @Override public Entry put(byte[] key, byte[] val) {
+    @Override public Entry put(byte[] key, byte[] bytes) {
         synchronized (mux) {
-            long crev = ++grev;
+            long curRev = ++rev;
+
+            long curUpdCntr = ++updCntr;
 
             // Update keysIdx.
             List<Long> revs = keysIdx.computeIfAbsent(key, k -> new ArrayList<>());
 
-            long lrev = revs.isEmpty() ? 0 : lastRevision(revs);
+            long lastRev = revs.isEmpty() ? 0 : lastRevision(revs);
 
-            revs.add(crev);
+            revs.add(curRev);
 
             // Update revsIdx.
-            NavigableMap<byte[], byte[]> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+            NavigableMap<byte[], Value> entries = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
+
+            Value val = new Value(bytes, curUpdCntr);
 
             entries.put(key, val);
 
-            revsIdx.put(crev, entries);
+            revsIdx.put(curRev, entries);
 
             // Return previous value.
-            if (lrev == 0)
+            if (lastRev == 0)
                 return Entry.empty(key);
 
-            NavigableMap<byte[], byte[]> lastVal = revsIdx.get(lrev);
+            NavigableMap<byte[], Value> lastRevVals = revsIdx.get(lastRev);
+
+            Value lastVal = lastRevVals.get(key);
 
-            Entry res = new Entry(key, lastVal.get(key), lrev);
+            Entry res = new Entry(key, lastVal.bytes(), lastRev, lastVal.updateCounter());
 
             //TODO: notify watchers
 
@@ -158,16 +170,18 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                             //return new AbstractMap.SimpleImmutableEntry<>(key, null);
                         }
 
-                        NavigableMap<byte[], byte[]> vals = revsIdx.get(rev);
+                        NavigableMap<byte[], Value> vals = revsIdx.get(rev);
 
                         if (vals == null || vals.isEmpty()) {
                             throw new IllegalStateException("vals == null || vals.isEmpty()");
                             //return new AbstractMap.SimpleImmutableEntry<>(key, null);
                         }
 
-                        byte[] val = vals.get(key);
+                        Value val = vals.get(key);
 
-                        return val == TOMBSTONE ? Entry.tombstone(key, rev) : new Entry(key, val, rev);
+                        return val.tombstone() ?
+                                Entry.tombstone(key, rev, val.updateCounter()) :
+                                new Entry(key, val.bytes(), rev, val.updateCounter());
                     }
                 }
             };
@@ -178,7 +192,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
         synchronized (mux) {
             NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(LEXICOGRAPHIC_COMPARATOR);
 
-            NavigableMap<Long, NavigableMap<byte[], byte[]>> compactedRevsIdx = new TreeMap<>();
+            NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new TreeMap<>();
 
             keysIdx.forEach((key, revs) -> compactForKey(key, revs, compactedKeysIdx, compactedRevsIdx));
 
@@ -192,18 +206,18 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
             byte[] key,
             List<Long> revs,
             NavigableMap<byte[], List<Long>> compactedKeysIdx,
-            NavigableMap<Long, NavigableMap<byte[], byte[]>> compactedRevsIdx
+            NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx
     ) {
         Long lrev = lastRevision(revs);
 
-        NavigableMap<byte[], byte[]> kv = revsIdx.get(lrev);
+        NavigableMap<byte[], Value> kv = revsIdx.get(lrev);
 
-        byte[] lastVal = kv.get(key);
+        Value lastVal = kv.get(key);
 
-        if (lastVal != TOMBSTONE) {
+        if (!lastVal.tombstone()) {
             compactedKeysIdx.put(key, listOf(lrev));
 
-            NavigableMap<byte[], byte[]> compactedKv = compactedRevsIdx.computeIfAbsent(
+            NavigableMap<byte[], Value> compactedKv = compactedRevsIdx.computeIfAbsent(
                     lrev,
                     k -> new TreeMap<>(LEXICOGRAPHIC_COMPARATOR)
             );
@@ -227,17 +241,17 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
 
         long lrev = rev == LATEST_REV ? lastRevision(revs) : rev;
 
-        NavigableMap<byte[], byte[]> entries = revsIdx.get(lrev);
+        NavigableMap<byte[], Value> entries = revsIdx.get(lrev);
 
         if (entries == null || entries.isEmpty())
             return Entry.empty(key);
 
-        byte[] val = entries.get(key);
+        Value val = entries.get(key);
 
-        if (val == TOMBSTONE)
-            return Entry.tombstone(key, lrev);
+        if (val.tombstone())
+            return Entry.tombstone(key, lrev, val.updateCounter());
 
-        return new Entry(key, val , lrev);
+        return new Entry(key, val.bytes() , lrev, val.updateCounter());
     }
 
     private static boolean isPrefix(byte[] pref, byte[] term) {
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
new file mode 100644
index 0000000..250a5ea
--- /dev/null
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/Value.java
@@ -0,0 +1,27 @@
+package org.apache.ignite.internal.metastorage.server;
+
+import org.jetbrains.annotations.NotNull;
+
+public class Value {
+    public static final byte[] TOMBSTONE = new byte[0];
+
+    private final byte[] bytes;
+    private final long updCntr;
+
+    public Value(@NotNull byte[] bytes, long updCntr) {
+        this.bytes = bytes;
+        this.updCntr = updCntr;
+    }
+
+    public byte[] bytes() {
+        return bytes;
+    }
+
+    public long updateCounter() {
+        return updCntr;
+    }
+
+    boolean tombstone() {
+        return bytes == TOMBSTONE;
+    }
+}
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
index f7fb17e..eae76fd 100644
--- 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
@@ -31,11 +31,13 @@ class SimpleInMemoryKeyValueStorageTest {
         byte[] val2_2 = kv(2, 2);
 
         assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
 
         // Previous entry is empty.
         Entry emptyEntry = storage.put(key1, val1_1);
 
         assertEquals(1, storage.revision());
+        assertEquals(1, storage.updateCounter());
         assertTrue(emptyEntry.empty());
 
         // Entry with rev == 1.
@@ -46,12 +48,15 @@ class SimpleInMemoryKeyValueStorageTest {
         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.put(key2, val2_2);
 
         assertEquals(2, storage.revision());
+        assertEquals(2, storage.updateCounter());
         assertTrue(emptyEntry.empty());
 
         // Entry with rev == 2.
@@ -62,7 +67,9 @@ class SimpleInMemoryKeyValueStorageTest {
         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.put(key1, val1_3);
@@ -72,7 +79,9 @@ class SimpleInMemoryKeyValueStorageTest {
         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);
@@ -82,7 +91,9 @@ class SimpleInMemoryKeyValueStorageTest {
         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.remove(key2);
@@ -92,7 +103,9 @@ class SimpleInMemoryKeyValueStorageTest {
         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.remove(key2);
@@ -100,11 +113,13 @@ class SimpleInMemoryKeyValueStorageTest {
         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.remove(key2).empty());
         assertTrue(storage.get(key2).empty());
 
@@ -116,7 +131,9 @@ class SimpleInMemoryKeyValueStorageTest {
         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.remove(key1);
@@ -124,11 +141,13 @@ class SimpleInMemoryKeyValueStorageTest {
         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.remove(key1).empty());
         assertTrue(storage.get(key1).empty());
     }
@@ -136,33 +155,40 @@ class SimpleInMemoryKeyValueStorageTest {
     @Test
     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.remove(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));
 
@@ -171,6 +197,7 @@ class SimpleInMemoryKeyValueStorageTest {
         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));
 
@@ -180,6 +207,7 @@ class SimpleInMemoryKeyValueStorageTest {
         assertArrayEquals(kv(2,2), e2.value());
         assertTrue(storage.get(k(2), 2).empty());
         assertEquals(3, e2.revision());
+        assertEquals(3, e2.updateCounter());
 
         Entry e3 = storage.get(k(3));
 
@@ -200,6 +228,7 @@ class SimpleInMemoryKeyValueStorageTest {
         fill("zoo", storage, expZooMap);
 
         assertEquals(300, storage.revision());
+        assertEquals(300, storage.updateCounter());
 
         assertIterate("key", storage, expKeyMap);
         assertIterate("zoo", storage, expZooMap);