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 2022/02/08 11:00:00 UTC

[GitHub] [ignite-3] denis-chudov commented on a change in pull request #631: IGNITE-16368 Implement a futures for causality tokens

denis-chudov commented on a change in pull request #631:
URL: https://github.com/apache/ignite-3/pull/631#discussion_r801442458



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */

Review comment:
       ```suggestion
       /** Versioned value storage. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.

Review comment:
       ```suggestion
        * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, Consumer, int)}.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);
+
+            return previousFut == null ? fut : previousFut;
+        }
+    }
+
+    /**
+     * Save the version of the value associated with the given causality token. If someone has got a future to await the value associated
+     * with the given causality token (see {@link #get(long)}, then the future will be completed.
+     *
+     * @param causalityToken Causality token.
+     * @param value          Current value.
+     */
+    public void set(long causalityToken, T value) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token earlier than actual [token={}, actual={}]",
+                causalityToken, actualToken0);
+
+        assert actualToken0 + 1 == causalityToken : IgniteStringFormatter.format(
+                "Previous token did not complete [token={}, previous={}]", causalityToken, causalityToken - 1);
+
+        CompletableFuture<T> res = history.putIfAbsent(causalityToken, CompletableFuture.completedFuture(value));
+
+        if (res == null || res.isCompletedExceptionally()) {
+            return;
+        }
+
+        assert !res.isDone() : IgniteStringFormatter.format("Different values asosiated with the token "
+                + "[token={}, value={}, prevValeu={}]", causalityToken, value, res.join());
+
+        res.complete(value);
+    }
+
+    /**
+     * Should be called on a storage revision update. This also triggers completion of a future created for the given causality token. It
+     * implies that all possible updates associated with this token have been already applied to the component.
+     *
+     * @param causalityToken Causality token.
+     */
+    private void onStorageRevisionUpdate(long causalityToken) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format(
+                "New token shoul be more than current [current={}, new={}]", actualToken0, causalityToken);

Review comment:
       ```suggestion
                   "New token should be greater than current [current={}, new={}]", actualToken0, causalityToken);
   ```
   Also I suppose that the new token must be greater that actual by exactly 1

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/causality/VersionValueTest.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.causality;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrowsExactly;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests of causality token implementation based on versioned value.
+ * {@link VersionedValue}
+ */
+public class VersionValueTest {

Review comment:
       I suggest also to add a test with combination of multiple VersionedValues like it was described in design examples.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);

Review comment:
       What if following happens:
   1. Thread1 enters #get, finds out that the given causality token is greater than actualToken and enters `else` section
   2. Thread2 makes multiple revision updates and actualToken becomes greater than causality token given for Thread1, it also trims history
   3. Thread1 puts new future to the history with an obsolete token
   
   Seems that Thread1 will receive a future that will never complete. This can lead to rare but misterious bugs

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);

Review comment:
       ```suggestion
               assert entry != null : IgniteStringFormatter.format("No future for token [token={}]", token);
   ```
   This also obliges the VersionedValue to be initialized with some value before the first revision update. We should mention it in javadoc.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {

Review comment:
       The logic of future completion that is in this default closure is necessary for normal work, in case when user defines some closure without setting the value inside of it. I suggest to move it to #onStorageRevisionUpdate method after calling user closure, so that we could complete the future if user didn't.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);

Review comment:
       ```suggestion
                           "No future for token [token={}]", token);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.

Review comment:
       ```suggestion
        * Constructor with default history size that equals 2 and no closure. See {@link #VersionedValue(BiConsumer, Consumer, int)}.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);
+
+            return previousFut == null ? fut : previousFut;
+        }
+    }
+
+    /**
+     * Save the version of the value associated with the given causality token. If someone has got a future to await the value associated
+     * with the given causality token (see {@link #get(long)}, then the future will be completed.
+     *
+     * @param causalityToken Causality token.
+     * @param value          Current value.
+     */
+    public void set(long causalityToken, T value) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token earlier than actual [token={}, actual={}]",

Review comment:
       ```suggestion
           assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token must be greater than actual by exactly 1 [token={}, actual={}]",
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);
+
+            return previousFut == null ? fut : previousFut;
+        }
+    }
+
+    /**
+     * Save the version of the value associated with the given causality token. If someone has got a future to await the value associated
+     * with the given causality token (see {@link #get(long)}, then the future will be completed.
+     *
+     * @param causalityToken Causality token.
+     * @param value          Current value.
+     */
+    public void set(long causalityToken, T value) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token earlier than actual [token={}, actual={}]",
+                causalityToken, actualToken0);
+
+        assert actualToken0 + 1 == causalityToken : IgniteStringFormatter.format(
+                "Previous token did not complete [token={}, previous={}]", causalityToken, causalityToken - 1);

Review comment:
       maybe we can unify this assert with previous one?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);
+
+            return previousFut == null ? fut : previousFut;
+        }
+    }
+
+    /**
+     * Save the version of the value associated with the given causality token. If someone has got a future to await the value associated
+     * with the given causality token (see {@link #get(long)}, then the future will be completed.
+     *
+     * @param causalityToken Causality token.
+     * @param value          Current value.
+     */
+    public void set(long causalityToken, T value) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token earlier than actual [token={}, actual={}]",
+                causalityToken, actualToken0);
+
+        assert actualToken0 + 1 == causalityToken : IgniteStringFormatter.format(
+                "Previous token did not complete [token={}, previous={}]", causalityToken, causalityToken - 1);
+
+        CompletableFuture<T> res = history.putIfAbsent(causalityToken, CompletableFuture.completedFuture(value));
+
+        if (res == null || res.isCompletedExceptionally()) {
+            return;
+        }
+
+        assert !res.isDone() : IgniteStringFormatter.format("Different values asosiated with the token "
+                + "[token={}, value={}, prevValeu={}]", causalityToken, value, res.join());

Review comment:
       ```suggestion
           assert !res.isDone() : IgniteStringFormatter.format("Different values associated with the token "
                   + "[token={}, value={}, prevValue={}]", causalityToken, value, res.join());
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);

Review comment:
       ```suggestion
           this(onStorageRevisionUpdate, observableRevisionUpdater, 2);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     * @param historySize               Size of the history of changes to store, including last applied token.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater,
+            int historySize
+    ) {
+        this.onStorageRevisionUpdate = onStorageRevisionUpdate == null ? (versionedValue, token) -> {
+            Entry<Long, CompletableFuture<T>> entry = history.floorEntry(token);
+
+            assert entry != null : IgniteStringFormatter.format("No future by token [token={}]", token);
+
+            if (!entry.getValue().isDone()) {
+                Entry<Long, CompletableFuture<T>> entryBefore = history.headMap(token).lastEntry();
+
+                assert entryBefore != null && entryBefore.getValue().isDone() : IgniteStringFormatter.format(
+                        "No future by token [token={}]", token);
+
+                entryBefore.getValue().whenComplete((t, throwable) -> {
+                    if (throwable != null) {
+                        entry.getValue().completeExceptionally(throwable);
+                    } else {
+                        entry.getValue().complete(t);
+                    }
+                });
+            }
+        } : onStorageRevisionUpdate;
+
+        observableRevisionUpdater.accept(this::onStorageRevisionUpdate);
+
+        this.historySize = historySize;
+    }
+
+    /**
+     * Constructor with default history size that equals 2. See {@link #VersionedValue(BiConsumer, int)}.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(
+            @Nullable BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate,
+            Consumer<Consumer<Long>> observableRevisionUpdater
+    ) {
+        this(null, observableRevisionUpdater, 2);
+    }
+
+    /**
+     * Constructor with default history size that equals 2 and no closure.
+     *
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.
+     */
+    public VersionedValue(Consumer<Consumer<Long>> observableRevisionUpdater) {
+        this(null, observableRevisionUpdater);
+    }
+
+    /**
+     * Creates a future for this value and causality token, or returns it if it already exists.
+     *
+     * <p>The returned future is associated with an update having the given causality token and completes when this update is finished
+     * applying.
+     *
+     * @param causalityToken Causality token. Let's assume that the update associated with token N is already applied to this value. Then,
+     *                       if token N is given as an argument, a completed future will be returned. If token N - 1 is given, this method
+     *                       returns the result in the state that is actual for the given token. If the token is strongly outdated, {@link
+     *                       OutdatedTokenException} is thrown. If token N + 1 is given, this method will return a future that will be
+     *                       completed when the update associated with token N + 1 will have been applied. Tokens that greater than N by
+     *                       more than 1 should never be passed.
+     * @return The future.
+     * @throws OutdatedTokenException If outdated token is passed as an argument.
+     */
+    public CompletableFuture<T> get(long causalityToken) throws OutdatedTokenException {
+        long actualToken0 = actualToken;
+
+        if (causalityToken <= actualToken0) {
+            Entry<Long, CompletableFuture<T>> histEntry = history.floorEntry(causalityToken);
+
+            if (histEntry == null) {
+                throw new OutdatedTokenException(causalityToken, actualToken0, historySize);
+            }
+
+            return histEntry.getValue();
+        } else {
+            var fut = new CompletableFuture<T>();
+
+            CompletableFuture<T> previousFut = history.putIfAbsent(causalityToken, fut);
+
+            return previousFut == null ? fut : previousFut;
+        }
+    }
+
+    /**
+     * Save the version of the value associated with the given causality token. If someone has got a future to await the value associated
+     * with the given causality token (see {@link #get(long)}, then the future will be completed.
+     *
+     * @param causalityToken Causality token.
+     * @param value          Current value.
+     */
+    public void set(long causalityToken, T value) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format("Token earlier than actual [token={}, actual={}]",
+                causalityToken, actualToken0);
+
+        assert actualToken0 + 1 == causalityToken : IgniteStringFormatter.format(
+                "Previous token did not complete [token={}, previous={}]", causalityToken, causalityToken - 1);
+
+        CompletableFuture<T> res = history.putIfAbsent(causalityToken, CompletableFuture.completedFuture(value));
+
+        if (res == null || res.isCompletedExceptionally()) {
+            return;
+        }
+
+        assert !res.isDone() : IgniteStringFormatter.format("Different values asosiated with the token "
+                + "[token={}, value={}, prevValeu={}]", causalityToken, value, res.join());
+
+        res.complete(value);
+    }
+
+    /**
+     * Should be called on a storage revision update. This also triggers completion of a future created for the given causality token. It
+     * implies that all possible updates associated with this token have been already applied to the component.
+     *
+     * @param causalityToken Causality token.
+     */
+    private void onStorageRevisionUpdate(long causalityToken) {
+        long actualToken0 = actualToken;
+
+        assert causalityToken > actualToken0 : IgniteStringFormatter.format(
+                "New token shoul be more than current [current={}, new={}]", actualToken0, causalityToken);
+
+        onStorageRevisionUpdate.accept(this, causalityToken);
+
+        if (history.size() > 1 && causalityToken - history.firstKey() >= historySize) {
+            trimToSize(causalityToken);
+        }
+
+        Entry<Long, CompletableFuture<T>> entry = history.floorEntry(causalityToken);
+
+        assert entry != null && entry.getValue().isDone() : IgniteStringFormatter.format(
+                "Future for the token is not completed [token={}]", causalityToken);
+
+        actualToken = causalityToken;
+    }
+
+    /**
+     * Trims the storage to history size.
+     *
+     * @param causalityToken Last token which was happend.

Review comment:
       ```suggestion
        * @param causalityToken Last token which is being applied.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/OutdatedTokenException.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.causality;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * This exception is thrown when {@link VersionedValue#get(long)} is called with an outdated token
+ * (this means that the history size of VersionedValue is not enough in order to get a value related to the token).
+ *
+ * <p>{@link VersionedValue} stores a value per the causality token.
+ * See {@link VersionedValue#get(long)}.
+ */
+public class OutdatedTokenException extends IgniteInternalCheckedException {
+
+    /**
+     * Constructor.
+     *
+     * @param outdatedToken The tokent which is expired.

Review comment:
       ```suggestion
        * @param outdatedToken The token which has expired.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/causality/VersionedValue.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.causality;
+
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parametrized type to store several versions of the value.
+ * A value can be available through the causality token, which is represented by long.
+ *
+ * @param <T> Type of real value.
+ */
+public class VersionedValue<T> {
+    /** Last applied casualty token. */
+    private volatile long actualToken;
+
+    /** Size of stored history. */
+    private final int historySize;
+
+    /** Closure applied on storage revision update. */
+    private final BiConsumer<VersionedValue<T>, Long> onStorageRevisionUpdate;
+
+    /** Bersoned value storage. */
+    private final ConcurrentNavigableMap<Long, CompletableFuture<T>> history = new ConcurrentSkipListMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param onStorageRevisionUpdate   Closure applied on storage revision update (see {@link #onStorageRevisionUpdate(long)}.
+     * @param observableRevisionUpdater A closure intended to connect this VersionedValue with a revision updater, that this VersionedValue
+     *                                  should be able to listen to, for receiving storage revision updates. This closure is called once on
+     *                                  a construction of this VersionedValue and accepts a {@link Consumer<Long>} that should be called on
+     *                                  every update of storage revision as a listener.

Review comment:
       ```suggestion
        *                                  every update of storage revision as a listener. IMPORTANT: Revision update shouldn't happen
        *                                  concurrently with {@link #set(long, T)} operations.                                  
   ```




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