You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "sergeyuttsel (via GitHub)" <gi...@apache.org> on 2023/06/20 14:30:11 UTC

[GitHub] [ignite-3] sergeyuttsel opened a new pull request, #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

sergeyuttsel opened a new pull request, #2227:
URL: https://github.com/apache/ignite-3/pull/2227

   https://issues.apache.org/jira/browse/IGNITE-19745


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238111679


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.

Review Comment:
   > If the lower bound is not compacted, then the upper bound is not compacted too.
   That's not true.



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.

Review Comment:
   > If the lower bound is not compacted, then the upper bound is not compacted too.
   
   That's not true.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236757037


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java:
##########
@@ -70,6 +70,16 @@ public interface KeyValueStorage extends ManuallyCloseable {
      */
     Entry get(byte[] key, long revUpperBound);
 
+    /**
+     * Returns an entries by the given key and bounded by given revisions.

Review Comment:
   Fixed



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,54 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGetEntries(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.
+
+        long[] revs;
+
+        try {
+            revs = getRevisions(key);
+        } catch (RocksDBException e) {
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
+        }
+
+        if (revs.length == 0) {
+            return Collections.emptyList();
+        }
+
+        long firstRev = minRevision(revs, revLowerBound);

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238107936


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -60,6 +61,13 @@ public interface MetaStorageManager extends IgniteComponent {
      */
     CompletableFuture<Entry> get(ByteArray key, long revUpperBound);
 
+    /**
+     * Retrieves entries for the given key and bound revisions.

Review Comment:
   Please add detailed javadoc. 
   -  boundaries included/excluded?
   - Result is a list of values ordered by revisions, isn't it?
   etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238293000


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -60,6 +61,13 @@ public interface MetaStorageManager extends IgniteComponent {
      */
     CompletableFuture<Entry> get(ByteArray key, long revUpperBound);
 
+    /**
+     * Retrieves entries for the given key and bound revisions.

Review Comment:
   I've updated javadoc. I tried to describe it more clearly. Not sure if I succeeded. But I can't write better.



##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.
+
+        long[] revs;
+
+        try {
+            revs = getRevisions(key);
+        } catch (RocksDBException e) {
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
+        }
+
+        if (revs.length == 0) {
+            return Collections.emptyList();
+        }
+
+        int firstRev = minRevisionIndex(revs, revLowerBound);

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238334631


##########
modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java:
##########
@@ -646,6 +655,40 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 throw CompactedException if revLowerBound is compacted.
+
+        List<Long> revs = keysIdx.get(key);
+
+        if (revs == null || revs.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        long firstRev = minRevision(revs, revLowerBound);
+        long lastRev = maxRevision(revs, revUpperBound);
+
+        // firstRev can be -1 if minRevision return -1. lastRev can be -1 if maxRevision return -1.
+        if (firstRev == -1 || lastRev == -1) {
+            return Collections.emptyList();
+        }
+
+        List<Entry> entries = new ArrayList<>();
+
+        for (int i = 0; i < revs.size(); i++) {
+            long rev = revs.get(i);
+
+            if (rev >= firstRev && rev <= lastRev) {

Review Comment:
   Sorry. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238292657


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.

Review Comment:
   I've updated javadoc. I tried to describe it more clearly. Not sure if I succeeded. But I can't write better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238089266


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.

Review Comment:
   It's not only about revLowerBound.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236521868


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -60,6 +61,13 @@ public interface MetaStorageManager extends IgniteComponent {
      */
     CompletableFuture<Entry> get(ByteArray key, long revUpperBound);
 
+    /**
+     * Retrieves entries for the given key and bound revisions.
+     * TODO: IGNITE-19735 move this method to another interface for interaction with local KeyValueStorage.
+     */
+    @Deprecated
+    List<Entry> getEntriesLocally(byte[] key, long revLowerBound, long revUpperBound);

Review Comment:
   Let's be consistent with other methods naming. There are bunch of method in `MetaStorageManager`, `RocksDbKeyValueStorage` and other related places that return entries, however neither of them have entries in it's name, you on the contrary introduces getEntriesLocally. Please rename it and all related `getEntries`, `doGetEntries`, etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238105469


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.

Review Comment:
   Just value? It's a list of revision based values ordered by revisions, isn't it. Please also address it in
   `Gets the value by key and revisions.`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236949433


##########
modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java:
##########
@@ -60,6 +61,13 @@ public interface MetaStorageManager extends IgniteComponent {
      */
     CompletableFuture<Entry> get(ByteArray key, long revUpperBound);
 
+    /**
+     * Retrieves entries for the given key and bound revisions.
+     * TODO: IGNITE-19735 move this method to another interface for interaction with local KeyValueStorage.
+     */
+    @Deprecated
+    List<Entry> getEntriesLocally(byte[] key, long revLowerBound, long revUpperBound);

Review Comment:
   Fixed. As we discussed I used a name `get` instead of `getEntries`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236757755


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +192,67 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getEntriesWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        assertEquals(9, storage.revision());
+        assertEquals(9, storage.updateCounter());
+
+        List<Entry> entries1 = storage.getEntries(key1, 2, 5);

Review Comment:
   I've added comments and some asserts in the test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238118058


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +191,88 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        removeFromMs(key1);
+
+        assertEquals(10, storage.revision());
+        assertEquals(10, storage.updateCounter());
+
+        // Get entries with the lower revision and the upper revision are bound to the key.
+        // One entry from the revision range is bound to another key.

Review Comment:
   It's still unclear.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236604998


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +192,67 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getEntriesWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        assertEquals(9, storage.revision());
+        assertEquals(9, storage.updateCounter());
+
+        List<Entry> entries1 = storage.getEntries(key1, 2, 5);
+        List<byte[]> values1 = entries1.stream().map(entry -> entry.value()).collect(Collectors.toList());
+
+        assertEquals(3, entries1.size());
+        assertTrue(values1.stream().anyMatch(e -> Arrays.equals(val2, e)));

Review Comment:
   Result collection is supposed to be ordered, so you should check that it contains all entries ordered by revision.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238290409


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +192,67 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getEntriesWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        assertEquals(9, storage.revision());
+        assertEquals(9, storage.updateCounter());
+
+        List<Entry> entries1 = storage.getEntries(key1, 2, 5);

Review Comment:
   I've updated comments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc merged pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc merged PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236521082


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java:
##########
@@ -319,6 +321,19 @@ public CompletableFuture<Entry> get(ByteArray key, long revUpperBound) {
         }
     }
 
+    @Override
+    public List<Entry> getEntriesLocally(byte[] key, long revLowerBound, long revUpperBound) {

Review Comment:
   Let's be consistent with other methods naming. There are bunch of method in `MetaStorageManager`, `RocksDbKeyValueStorage` and other related places that return entries, however neither of them have entries in it's name, you on the contrary introduces getEntriesLocally. Please rename it and all related `getEntries`, `doGetEntries`, etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236531427


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,54 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGetEntries(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.
+
+        long[] revs;
+
+        try {
+            revs = getRevisions(key);
+        } catch (RocksDBException e) {
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
+        }
+
+        if (revs.length == 0) {
+            return Collections.emptyList();
+        }
+
+        long firstRev = minRevision(revs, revLowerBound);

Review Comment:
   I'd rather introduce a pair of new methods minRevision**Index** maxRevision**Index** in order to eliminate following checks `rev >= firstRev && rev <= lastRev` and just iterate from firstRevisionIndex to lastRevisionIndex.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238319474


##########
modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java:
##########
@@ -646,6 +655,40 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 throw CompactedException if revLowerBound is compacted.
+
+        List<Long> revs = keysIdx.get(key);
+
+        if (revs == null || revs.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        long firstRev = minRevision(revs, revLowerBound);
+        long lastRev = maxRevision(revs, revUpperBound);
+
+        // firstRev can be -1 if minRevision return -1. lastRev can be -1 if maxRevision return -1.
+        if (firstRev == -1 || lastRev == -1) {
+            return Collections.emptyList();
+        }
+
+        List<Entry> entries = new ArrayList<>();
+
+        for (int i = 0; i < revs.size(); i++) {
+            long rev = revs.get(i);
+
+            if (rev >= firstRev && rev <= lastRev) {

Review Comment:
   I'd also use firstRevisionIndex/lastRevisionIndex as in Rocks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238113967


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1252,6 +1308,46 @@ private static long maxRevision(long[] revs, long upperBoundRev) {
         return -1;
     }
 
+    /**
+     * Returns index of minimum revision which must be greater or equal to {@code lowerBoundRev}.

Review Comment:
   So the **index** should be greater or equal to {@code lowerBoundRev}, is that correct? ;)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236524093


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java:
##########
@@ -70,6 +70,16 @@ public interface KeyValueStorage extends ManuallyCloseable {
      */
     Entry get(byte[] key, long revUpperBound);
 
+    /**
+     * Returns an entries by the given key and bounded by given revisions.

Review Comment:
   an isn't suitable for the plural.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236601472


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +192,67 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getEntriesWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        assertEquals(9, storage.revision());
+        assertEquals(9, storage.updateCounter());
+
+        List<Entry> entries1 = storage.getEntries(key1, 2, 5);

Review Comment:
   It's difficult to understand which test scenarios are assumed here. Please add comments that will explain the details. Also seems that some tests are missing for example ones that will return empty list if there are no data in proposed revisions boundaries.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238101765


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.

Review Comment:
   Why? If the lower bound is compacted, then it doesn't matter if the upper bound is compacted or not.
   If the lower bound is not compacted, then the upper bound is not compacted too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238112862


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.
+
+        long[] revs;
+
+        try {
+            revs = getRevisions(key);
+        } catch (RocksDBException e) {
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
+        }
+
+        if (revs.length == 0) {
+            return Collections.emptyList();
+        }
+
+        int firstRev = minRevisionIndex(revs, revLowerBound);

Review Comment:
   It's not firstRev but firstRevIndex. Same for lastRev.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238293618


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +191,88 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        removeFromMs(key1);
+
+        assertEquals(10, storage.revision());
+        assertEquals(10, storage.updateCounter());
+
+        // Get entries with the lower revision and the upper revision are bound to the key.
+        // One entry from the revision range is bound to another key.

Review Comment:
   I've tried to describe it more clearly. Not sure if I succeeded. But I can't write better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1238283724


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java:
##########
@@ -1215,6 +1227,50 @@ private Entry doGet(byte[] key, long revUpperBound) {
         return doGetValue(key, lastRev);
     }
 
+    /**
+     * Gets the value by key and revisions.
+     *
+     * @param key            Target key.
+     * @param revLowerBound  Target lower bound of revision.
+     * @param revUpperBound  Target upper bound of revision.
+     * @return Value.
+     */
+    private List<Entry> doGet(byte[] key, long revLowerBound, long revUpperBound) {
+        assert revLowerBound >= 0 : "Invalid arguments: [revLowerBound=" + revLowerBound + ']';
+        assert revUpperBound >= 0 : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
+        assert revUpperBound >= revLowerBound
+                : "Invalid arguments: [revLowerBound=" + revLowerBound + ", revUpperBound=" + revUpperBound + ']';
+        // TODO: IGNITE-19782 assert that revLowerBound is not compacted.

Review Comment:
   My fault. You are right.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2227: IGNITE-19745 Added a method for local obtaining entries in MetaStorage from lower bound revision to upper bound revision.

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2227:
URL: https://github.com/apache/ignite-3/pull/2227#discussion_r1236758044


##########
modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/BasicOperationsKeyValueStorageTest.java:
##########
@@ -190,6 +192,67 @@ void getWithRevisionBound() {
         assertFalse(key3EntryBounded5.empty());
     }
 
+    @Test
+    void getEntriesWithRevisionLowerUpperBound() {
+        byte[] key1 = key(1);
+        byte[] key2 = key(2);
+
+        byte[] val1 = keyValue(1, 1);
+        byte[] val2 = keyValue(1, 2);
+        byte[] val3 = keyValue(2, 3);
+        byte[] val4 = keyValue(1, 4);
+        byte[] val5 = keyValue(1, 5);
+        byte[] val6 = keyValue(2, 6);
+        byte[] val7 = keyValue(2, 7);
+        byte[] val8 = keyValue(1, 8);
+        byte[] val9 = keyValue(1, 9);
+
+        assertEquals(0, storage.revision());
+        assertEquals(0, storage.updateCounter());
+
+        putToMs(key1, val1);
+        putToMs(key1, val2);
+        putToMs(key2, val3);
+        putToMs(key1, val4);
+        putToMs(key1, val5);
+        putToMs(key2, val6);
+        putToMs(key2, val7);
+        putToMs(key1, val8);
+        putToMs(key1, val9);
+
+        assertEquals(9, storage.revision());
+        assertEquals(9, storage.updateCounter());
+
+        List<Entry> entries1 = storage.getEntries(key1, 2, 5);
+        List<byte[]> values1 = entries1.stream().map(entry -> entry.value()).collect(Collectors.toList());
+
+        assertEquals(3, entries1.size());
+        assertTrue(values1.stream().anyMatch(e -> Arrays.equals(val2, e)));

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org