You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sd...@apache.org on 2023/06/28 14:51:34 UTC

[ignite-3] 01/01: IGNITE-19778 Initial

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

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

commit b3dcc174f20c75d839957cca3e6d7c53b3c30232
Author: Semyon Danilov <sa...@yandex.ru>
AuthorDate: Wed Jun 28 18:51:26 2023 +0400

    IGNITE-19778 Initial
---
 .../configuration/ConfigurationChanger.java        | 40 +++++-----
 .../storage/ConfigurationStorage.java              | 11 ---
 .../storage/TestConfigurationStorage.java          |  6 --
 .../java/org/apache/ignite/lang/ByteArray.java     |  4 +
 .../internal/metastorage/MetaStorageManager.java   |  3 +
 .../metastorage/impl/MetaStorageManagerImpl.java   | 32 ++++----
 .../server/persistence/RocksDbKeyValueStorage.java |  4 +-
 .../server/SimpleInMemoryKeyValueStorage.java      |  4 +-
 .../ItDistributedConfigurationStorageTest.java     |  1 -
 .../org/apache/ignite/internal/app/IgniteImpl.java | 10 +--
 .../storage/DistributedConfigurationStorage.java   | 88 ++++------------------
 .../storage/LocalConfigurationStorage.java         |  7 --
 .../storage/LocalFileConfigurationStorage.java     |  5 --
 13 files changed, 57 insertions(+), 158 deletions(-)

diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
index fa1e4bd7e6..6b115f5995 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationChanger.java
@@ -644,28 +644,24 @@ public abstract class ConfigurationChanger implements DynamicConfigurationChange
                     rwLock.writeLock().unlock();
                 }
 
-                // Save revisions for recovery.
-                return storage.writeConfigurationRevision(oldStorageRoots.version, newStorageRoots.version)
-                        .thenCompose(unused -> {
-                            long notificationNumber = notificationListenerCnt.incrementAndGet();
-
-                            CompletableFuture<Void> notificationFuture;
-
-                            if (dataValuesPrefixMap.isEmpty()) {
-                                notificationFuture = configurationUpdateListener.onRevisionUpdated(newChangeId, notificationNumber);
-                            } else {
-                                notificationFuture = configurationUpdateListener
-                                        .onConfigurationUpdated(oldSuperRoot, newSuperRoot, newChangeId, notificationNumber);
-                            }
-
-                            return notificationFuture.whenComplete((v, t) -> {
-                                if (t == null) {
-                                    oldStorageRoots.changeFuture.complete(null);
-                                } else {
-                                    oldStorageRoots.changeFuture.completeExceptionally(t);
-                                }
-                            });
-                        });
+                long notificationNumber = notificationListenerCnt.incrementAndGet();
+
+                CompletableFuture<Void> notificationFuture;
+
+                if (dataValuesPrefixMap.isEmpty()) {
+                    notificationFuture = configurationUpdateListener.onRevisionUpdated(newChangeId, notificationNumber);
+                } else {
+                    notificationFuture = configurationUpdateListener
+                            .onConfigurationUpdated(oldSuperRoot, newSuperRoot, newChangeId, notificationNumber);
+                }
+
+                return notificationFuture.whenComplete((v, t) -> {
+                    if (t == null) {
+                        oldStorageRoots.changeFuture.complete(null);
+                    } else {
+                        oldStorageRoots.changeFuture.completeExceptionally(t);
+                    }
+                });
             }
 
             @Override
diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/storage/ConfigurationStorage.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/storage/ConfigurationStorage.java
index e31f7d4e33..d9663eb284 100644
--- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/storage/ConfigurationStorage.java
+++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/storage/ConfigurationStorage.java
@@ -79,17 +79,6 @@ public interface ConfigurationStorage extends ManuallyCloseable {
      */
     CompletableFuture<Long> lastRevision();
 
-    /**
-     * Writes previous and current configuration's MetaStorage revision for recovery.
-     * We need previous and current for the fail-safety: in case if node fails before changing master key on configuration update,
-     * MetaStorage's applied revision will be lower than {@code currentRevision} and we will be using previous revision.
-     *
-     * @param prevRevision Previous revision.
-     * @param currentRevision Current revision.
-     * @return A future that will be completed when revisions are written to the storage.
-     */
-    CompletableFuture<Void> writeConfigurationRevision(long prevRevision, long currentRevision);
-
     /**
      * Closes the storage.
      */
diff --git a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/storage/TestConfigurationStorage.java b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/storage/TestConfigurationStorage.java
index 37aa61facc..4231915b31 100644
--- a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/storage/TestConfigurationStorage.java
+++ b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/storage/TestConfigurationStorage.java
@@ -170,12 +170,6 @@ public class TestConfigurationStorage implements ConfigurationStorage {
         return CompletableFuture.completedFuture(version);
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, long currentRevision) {
-        return CompletableFuture.completedFuture(null);
-    }
-
     /**
      * Increase the current revision of the storage.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java b/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
index 76129b6d55..e8900917aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/ByteArray.java
@@ -77,6 +77,10 @@ public final class ByteArray implements Comparable<ByteArray> {
         return arr;
     }
 
+    public int length() {
+        return arr.length;
+    }
+
     /** {@inheritDoc} */
     @Override
     public boolean equals(Object o) {
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index a862817aef..8451acf691 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.metastorage.dsl.StatementResult;
 import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
 import org.apache.ignite.internal.metastorage.exceptions.OperationTimeoutException;
 import org.apache.ignite.internal.metastorage.server.time.ClusterTime;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.jetbrains.annotations.Nullable;
@@ -86,6 +87,8 @@ public interface MetaStorageManager extends IgniteComponent {
      */
     Entry getLocally(byte[] key, long revUpperBound);
 
+    Cursor<Entry> getLocally(ByteArray startKey, ByteArray endKey, long revision);
+
     /**
      * Looks up a timestamp by a revision. This should only be invoked if it is guaranteed that the
      * revision is available in the local storage. This method always operates locally.
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
index 3d3702dbaa..532fea80aa 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.raft.RaftManager;
 import org.apache.ignite.internal.raft.RaftNodeDisruptorConfiguration;
 import org.apache.ignite.internal.raft.RaftNodeId;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.vault.VaultEntry;
@@ -171,7 +172,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
 
                 assert targetRevision != null;
 
-                listenForRecovery(recoveryFinishedFuture, targetRevision);
+                listenForRecovery(targetRevision);
             });
 
             return recoveryFinishedFuture;
@@ -180,10 +181,10 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         }
     }
 
-    private void listenForRecovery(CompletableFuture<Long> res, long targetRevision) {
+    private void listenForRecovery(long targetRevision) {
         storage.setRecoveryRevisionListener(storageRevision -> {
             if (!busyLock.enterBusy()) {
-                res.completeExceptionally(new NodeStoppingException());
+                recoveryFinishedFuture.completeExceptionally(new NodeStoppingException());
 
                 return;
             }
@@ -195,7 +196,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
 
                 storage.setRecoveryRevisionListener(null);
 
-                if (res.complete(targetRevision)) {
+                if (recoveryFinishedFuture.complete(targetRevision)) {
                     LOG.info("Finished MetaStorage recovery");
                 }
             } finally {
@@ -204,7 +205,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         });
 
         if (!busyLock.enterBusy()) {
-            res.completeExceptionally(new NodeStoppingException());
+            recoveryFinishedFuture.completeExceptionally(new NodeStoppingException());
 
             return;
         }
@@ -214,7 +215,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
             if (storage.revision() >= targetRevision) {
                 storage.setRecoveryRevisionListener(null);
 
-                if (res.complete(targetRevision)) {
+                if (recoveryFinishedFuture.complete(targetRevision)) {
                     LOG.info("Finished MetaStorage recovery");
                 }
             }
@@ -425,6 +426,11 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         }
     }
 
+    @Override
+    public Cursor<Entry> getLocally(ByteArray startKey, ByteArray endKey, long revision) {
+        return storage.range(startKey.bytes(), endKey.bytes(), revision);
+    }
+
     @Override
     public HybridTimestamp timestampByRevision(long revision) {
         if (!busyLock.enterBusy()) {
@@ -735,19 +741,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
         clusterTime.updateSafeTime(time);
 
         try {
-            CompletableFuture<Void> saveToVaultFuture;
-
-            if (watchEvent.entryEvents().isEmpty()) {
-                saveToVaultFuture = vaultMgr.put(APPLIED_REV_KEY, longToBytes(watchEvent.revision()));
-            } else {
-                Map<ByteArray, byte[]> batch = IgniteUtils.newHashMap(watchEvent.entryEvents().size() + 1);
-
-                batch.put(APPLIED_REV_KEY, longToBytes(watchEvent.revision()));
-
-                watchEvent.entryEvents().forEach(e -> batch.put(new ByteArray(e.newEntry().key()), e.newEntry().value()));
-
-                saveToVaultFuture = vaultMgr.putAll(batch);
-            }
+            CompletableFuture<Void> saveToVaultFuture = vaultMgr.put(APPLIED_REV_KEY, longToBytes(watchEvent.revision()));
 
             return saveToVaultFuture.thenRun(() -> appliedRevision = watchEvent.revision());
         } finally {
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
index 08ff0c2cb7..2bc2c7f05f 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
@@ -1495,9 +1495,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
     }
 
     private void replayUpdates(long lowerRevision, long upperRevision) {
-        // TODO: https://issues.apache.org/jira/browse/IGNITE-19778 Should be Math.max, so we start from the revision that
-        // components restored their state to (lowerRevision).
-        long minWatchRevision = Math.min(lowerRevision, watchProcessor.minWatchRevision().orElse(-1));
+        long minWatchRevision = Math.max(lowerRevision, watchProcessor.minWatchRevision().orElse(-1));
 
         if (minWatchRevision == -1 || minWatchRevision > upperRevision) {
             // No events to replay, we can start processing more recent events from the event queue.
diff --git a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
index 2461c4bece..b8cb0d3508 100644
--- a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
+++ b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -492,9 +492,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
     }
 
     private void replayUpdates(long startRevision) {
-        // TODO: https://issues.apache.org/jira/browse/IGNITE-19778 Should be Math.max, so we start from the revision that
-        // components restored their state to (lowerRevision).
-        long minWatchRevision = Math.min(startRevision, watchProcessor.minWatchRevision().orElse(-1));
+        long minWatchRevision = Math.max(startRevision, watchProcessor.minWatchRevision().orElse(-1));
 
         if (minWatchRevision <= 0) {
             return;
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
index 89dfa81caa..59facf67b4 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
@@ -212,7 +212,6 @@ public class ItDistributedConfigurationStorageTest {
             node.waitWatches();
 
             assertThat(node.cfgStorage.write(data, 0), willBe(equalTo(true)));
-            assertThat(node.cfgStorage.writeConfigurationRevision(0, 1), willCompleteSuccessfully());
 
             assertTrue(waitForCondition(
                     () -> node.metaStorageManager.appliedRevision() != 0,
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 32eb73e031..c35e246cd3 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -99,8 +99,6 @@ import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.raft.client.TopologyAwareRaftGroupServiceFactory;
 import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
 import org.apache.ignite.internal.raft.storage.impl.VolatileLogStorageFactoryCreator;
-import org.apache.ignite.internal.recovery.ConfigurationCatchUpListener;
-import org.apache.ignite.internal.recovery.RecoveryCompletionFutureFactory;
 import org.apache.ignite.internal.replicator.ReplicaManager;
 import org.apache.ignite.internal.replicator.ReplicaService;
 import org.apache.ignite.internal.rest.RestComponent;
@@ -748,16 +746,10 @@ public class IgniteImpl implements Ignite {
                     .thenComposeAsync(v -> {
                         LOG.info("Components started, performing recovery");
 
-                        // Recovery future must be created before configuration listeners are triggered.
-                        CompletableFuture<?> recoveryFuture = RecoveryCompletionFutureFactory.create(
-                                clusterCfgMgr,
-                                fut -> new ConfigurationCatchUpListener(cfgStorage, fut, LOG)
-                        );
-
                         return notifyConfigurationListeners()
                                 .thenComposeAsync(t -> {
                                     // Deploy all registered watches because all components are ready and have registered their listeners.
-                                    return metaStorageMgr.deployWatches().thenCompose(unused -> recoveryFuture);
+                                    return metaStorageMgr.deployWatches();
                                 }, startupExecutor);
                     }, startupExecutor)
                     .thenRunAsync(() -> {
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
index f0dd8e74c1..102b966191 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
@@ -50,10 +50,8 @@ import org.apache.ignite.internal.thread.NamedThreadFactory;
 import org.apache.ignite.internal.util.ByteUtils;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.internal.vault.VaultEntry;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Distributed configuration storage.
@@ -70,11 +68,6 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
      */
     private static final ByteArray MASTER_KEY = new ByteArray(DISTRIBUTED_PREFIX + "$master$key");
 
-    /**
-     * Vault's key for a value of previous and current configuration's MetaStorage revision.
-     */
-    private static final ByteArray CONFIGURATION_REVISIONS_KEY = new ByteArray("$revisions");
-
     /**
      * Prefix for all keys in the distributed storage. This key is expected to be the first key in lexicographical order of distributed
      * configuration keys.
@@ -91,9 +84,6 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
     /** Meta storage manager. */
     private final MetaStorageManager metaStorageMgr;
 
-    /** Vault manager. */
-    private final VaultManager vaultMgr;
-
     /** Configuration changes listener. */
     private volatile ConfigurationStorageListener lsnr;
 
@@ -127,8 +117,6 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
      */
     public DistributedConfigurationStorage(MetaStorageManager metaStorageMgr, VaultManager vaultMgr) {
         this.metaStorageMgr = metaStorageMgr;
-
-        this.vaultMgr = vaultMgr;
     }
 
     @Override
@@ -206,60 +194,40 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
 
     @Override
     public CompletableFuture<Data> readDataOnRecovery() throws StorageException {
-        CompletableFuture<Data> future = vaultMgr.get(CONFIGURATION_REVISIONS_KEY)
-                .thenApplyAsync(entry -> {
-                    long revision = resolveRevision(metaStorageMgr.appliedRevision(), entry);
-
-                    return readDataOnRecovery0(revision);
-                }, threadPool);
+        CompletableFuture<Data> future = metaStorageMgr.recoveryFinishedFuture()
+                .thenApplyAsync(this::readDataOnRecovery0);
 
         return registerFuture(future);
     }
 
-    /**
-     * Resolves current configuration revision based on the saved in the Vault revision of the metastorage and also previous and current
-     * revisions of the configuration saved in the Vault.
-     *
-     * @param metaStorageRevision Meta Storage revision.
-     * @param revisionsEntry Configuration revisions entry.
-     * @return Configuration revision.
-     */
-    private static long resolveRevision(long metaStorageRevision, @Nullable VaultEntry revisionsEntry) {
-        if (revisionsEntry != null) {
-            byte[] value = revisionsEntry.value();
-            long prevMasterKeyRevision = ByteUtils.bytesToLong(value, 0);
-            long curMasterKeyRevision = ByteUtils.bytesToLong(value, Long.BYTES);
-
-            // If current master key revision is higher than applied revision, then node failed
-            // before applied revision changed, so we have to use previous master key revision
-            return curMasterKeyRevision <= metaStorageRevision ? curMasterKeyRevision : prevMasterKeyRevision;
-        } else {
-            // Configuration has not been updated yet, so it is safe to return 0 as the revision for the master key.
-            return 0L;
-        }
-    }
-
     private Data readDataOnRecovery0(long cfgRevision) {
         var data = new HashMap<String, Serializable>();
 
-        try (Cursor<VaultEntry> entries = storedDistributedConfigKeys()) {
-            for (VaultEntry entry : entries) {
-                ByteArray key = entry.key();
+        byte[] masterKey = MASTER_KEY.bytes();
+        boolean sawMasterKey = false;
+
+        try (Cursor<Entry> cursor = metaStorageMgr.getLocally(DST_KEYS_START_RANGE, DST_KEYS_END_RANGE, cfgRevision)) {
+            for (Entry entry : cursor) {
+                byte[] key = entry.key();
                 byte[] value = entry.value();
 
-                // vault iterator should not return nulls as values
+                // MetaStorage iterator should not return nulls as values.
                 assert value != null;
 
-                if (key.equals(MASTER_KEY)) {
+                if (!sawMasterKey && Arrays.equals(masterKey, key)) {
+                    sawMasterKey = true;
+
                     continue;
                 }
 
-                String dataKey = key.toString().substring(DISTRIBUTED_PREFIX.length());
+                byte[] keyWithoutPrefix = Arrays.copyOfRange(key, DST_KEYS_START_RANGE.length(), key.length);
+
+                var dataKey = new String(keyWithoutPrefix, UTF_8);
 
                 data.put(dataKey, ConfigurationSerializationUtil.fromBytes(value));
             }
         } catch (Exception e) {
-            throw new StorageException("Exception when closing a Vault cursor", e);
+            throw new StorageException("Exception reading data on recovery", e);
         }
 
         assert data.isEmpty() || cfgRevision > 0;
@@ -374,30 +342,6 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
         return metaStorageMgr.get(MASTER_KEY).thenApply(Entry::revision);
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, long currentRevision) {
-        byte[] value = new byte[Long.BYTES * 2];
-
-        ByteUtils.putLongToBytes(prevRevision, value, 0);
-        ByteUtils.putLongToBytes(currentRevision, value, Long.BYTES);
-
-        return vaultMgr.put(CONFIGURATION_REVISIONS_KEY, value);
-    }
-
-    /**
-     * Method that returns all distributed configuration keys from the meta storage that were stored in the vault filtered out by the
-     * current applied revision as an upper bound. Applied revision is a revision of the last successful vault update.
-     *
-     * <p>This is possible to distinguish cfg keys from meta storage because we add a special prefix {@link
-     * DistributedConfigurationStorage#DISTRIBUTED_PREFIX} to all configuration keys that we put to the meta storage.
-     *
-     * @return Iterator built upon all distributed configuration entries stored in vault.
-     */
-    private Cursor<VaultEntry> storedDistributedConfigKeys() {
-        return vaultMgr.range(DST_KEYS_START_RANGE, DST_KEYS_END_RANGE);
-    }
-
     /**
      * Increments the last character of the given string.
      */
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalConfigurationStorage.java
index e81129304c..c026a6aa04 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalConfigurationStorage.java
@@ -223,13 +223,6 @@ public class LocalConfigurationStorage implements ConfigurationStorage {
                 .thenApply(entry -> entry == null ? 0 : (Long) fromBytes(entry.value()));
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, long currentRevision) {
-        // No-op.
-        return CompletableFuture.completedFuture(null);
-    }
-
     /**
      * Increments the last character of the given string.
      */
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java
index 7afc824278..742745c4cb 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/LocalFileConfigurationStorage.java
@@ -219,11 +219,6 @@ public class LocalFileConfigurationStorage implements ConfigurationStorage {
         return CompletableFuture.completedFuture(lastRevision);
     }
 
-    @Override
-    public CompletableFuture<Void> writeConfigurationRevision(long prevRevision, long currentRevision) {
-        return CompletableFuture.completedFuture(null);
-    }
-
     @Override
     public void close() {
         futureTracker.cancelInFlightFutures();