You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2023/01/20 10:00:33 UTC

[GitHub] [ignite-3] sashapolo commented on a diff in pull request #1490: IGNITE-18397 Rework Watches based on Raft Learners

sashapolo commented on code in PR #1490:
URL: https://github.com/apache/ignite-3/pull/1490#discussion_r1082310535


##########
modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java:
##########
@@ -220,32 +222,54 @@ public interface KeyValueStorage extends ManuallyCloseable {
      * @param keyFrom Start key of range (inclusive).
      * @param keyTo   Last key of range (exclusive).
      * @param rev     Start revision number.
-     * @return Cursor by update events.
      */
-    Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev);
+    void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener);
 
     /**
-     * Creates subscription on updates of entries corresponding to the given keys range (where upper bound is unlimited) and starting from
-     * the given revision number.
+     * Registers a watch listener by a key prefix.
      *
-     * @param key Start key of range (inclusive).
-     * @param rev Start revision number.
-     * @return Cursor by update events.
+     * @param prefix Prefix to listen to.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
      */
-    Cursor<WatchEvent> watch(byte[] key, long rev);
+    void watchPrefix(byte[] prefix, long rev, WatchListener listener);
 
     /**
-     * Creates subscription on updates of entries corresponding to the given keys collection and starting from the given revision number.
+     * Registers a watch listener for the provided key.
      *
-     * @param keys Collection of keys
-     * @param rev  Start revision number.
-     * @return Cursor by update events.
+     * @param key Meta Storage key.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
      */
-    Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
+    void watchExact(byte[] key, long rev, WatchListener listener);
+
+    /**
+     * Registers a watch listener for the provided keys.
+     *
+     * @param keys Meta Storage keys.
+     * @param rev Starting Meta Storage revision.
+     * @param listener Listener which will be notified for each update.
+     */
+    void watchExact(Collection<byte[]> keys, long rev, WatchListener listener);
+
+    /**
+     * Starts all registered watches.
+     *
+     * <p>Before calling this method, watches will not receive any updates.
+     *
+     * @param revisionCallback Callback that will be invoked after all watches of a particular revision are processed, with the revision
+     *      as its argument.
+     */
+    void startWatches(LongConsumer revisionCallback);
+
+    /**
+     * Unregisters a watch listener.
+     */
+    void removeWatch(WatchListener listener);
 
     /**
      * Compacts storage (removes tombstones).
-     * TODO: IGNITE-16444 Сorrect compaction for Metastorage.
+     * TODO: IGNITE-16444 Correct compaction for Metastorage.

Review Comment:
   I think yes



-- 
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