You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ap...@apache.org on 2023/11/16 07:44:05 UTC

(ignite-3) branch main updated: IGNITE-20854 Configure idleSafeTimePropagationPeriod with a distributed property (#2840)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new f3ee9b206b IGNITE-20854 Configure idleSafeTimePropagationPeriod with a distributed property (#2840)
f3ee9b206b is described below

commit f3ee9b206bc8e4389d178f11da4eefba01264683
Author: Roman Puchkovskiy <ro...@gmail.com>
AuthorDate: Thu Nov 16 11:44:00 2023 +0400

    IGNITE-20854 Configure idleSafeTimePropagationPeriod with a distributed property (#2840)
---
 .../internal/catalog/CatalogManagerImpl.java       |  8 +--
 .../internal/catalog/CatalogManagerSelfTest.java   | 29 -----------
 .../testframework/BaseIgniteAbstractTest.java      |  2 -
 .../testframework/TestIgnitionManager.java         | 11 +++-
 modules/replicator/build.gradle                    |  4 ++
 .../ignite/internal/replicator/ReplicaManager.java | 13 -----
 .../ReplicationConfigurationModule.java}           | 26 +++++++---
 .../ReplicationConfigurationSchema.java            | 36 +++++++++++++
 .../ReplicationConfigurationModuleTest.java}       | 29 ++++++++---
 .../internal/readonly/ItReadOnlyTxInPastTest.java  |  7 +--
 .../runner/app/ItIgniteNodeRestartTest.java        |  9 ++--
 .../org/apache/ignite/internal/app/IgniteImpl.java | 16 ++++--
 .../ignite/internal/app/SameValueLongSupplier.java | 49 ++++++++++++++++++
 .../internal/app/SameValueLongSupplierTest.java    | 59 ++++++++++++++++++++++
 .../internal/ClusterPerTestIntegrationTest.java    |  7 ++-
 .../rebalance/ItRebalanceRecoveryTest.java         |  3 --
 .../apache/ignite/internal/tx/TxManagerTest.java   | 24 ++++-----
 17 files changed, 238 insertions(+), 94 deletions(-)

diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
index afd92ced19..3078ecae33 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
@@ -530,12 +530,14 @@ public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, Cata
 
     private static Catalog applyUpdateFinal(Catalog catalog, VersionedUpdate update, HybridTimestamp metaStorageUpdateTimestamp) {
         long activationTimestamp = metaStorageUpdateTimestamp.addPhysicalTime(update.delayDurationMs()).longValue();
-        long prevVersionActivationTimestamp = catalog.time() + 1;
+
+        assert activationTimestamp > catalog.time()
+                : "Activation timestamp " + activationTimestamp + " must be greater than previous catalog version activation timestamp "
+                        + catalog.time();
 
         return new Catalog(
                 update.version(),
-                // Remove this maxing when https://issues.apache.org/jira/browse/IGNITE-20854 is fixed and DelayDuration is truly constant.
-                Math.max(activationTimestamp, prevVersionActivationTimestamp),
+                activationTimestamp,
                 catalog.objectIdGenState(),
                 catalog.zones(),
                 catalog.schemas()
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
index ed41ce3795..4d1a3a3117 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
@@ -50,7 +50,6 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutur
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
 import static org.apache.ignite.sql.ColumnType.DECIMAL;
 import static org.apache.ignite.sql.ColumnType.INT32;
 import static org.apache.ignite.sql.ColumnType.INT64;
@@ -58,7 +57,6 @@ import static org.apache.ignite.sql.ColumnType.NULL;
 import static org.apache.ignite.sql.ColumnType.STRING;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.hasItems;
 import static org.hamcrest.Matchers.is;
@@ -1994,33 +1992,6 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertThat(fireEventFuture, willCompleteSuccessfully());
     }
 
-    @Test
-    public void activationTimeIsStrictlyMonotonic() {
-        // Prepare schema changes.
-        ColumnParams column = ColumnParams.builder().name("ID").type(INT32).build();
-        CatalogCommand cmd1 = BaseCatalogManagerTest.createTableCommand(TABLE_NAME, List.of(column), List.of("ID"), null);
-        CatalogCommand cmd2 = BaseCatalogManagerTest.createTableCommand("test2", List.of(column), List.of("ID"), null);
-
-        // Make first schema change with delay = 1000.
-        delayDuration.set(10_000);
-        CompletableFuture<Void> schemaChangeFuture0 = manager.execute(cmd1);
-
-        // Make second schema change with delay = 1.
-        delayDuration.set(1);
-        CompletableFuture<Void> schemaChangeFuture1 = manager.execute(cmd2);
-
-        // Move clock forward to avoid awaiting.
-        clock.update(clock.now().addPhysicalTime(11_000));
-
-        assertThat(schemaChangeFuture0, willSucceedFast());
-        assertThat(schemaChangeFuture1, willSucceedFast());
-
-        // Make sure that we are getting the latest version of the schema using current timestamp.
-        int latestVer = manager.latestCatalogVersion();
-        int currentTsVer = manager.activeCatalogVersion(clock.now().longValue());
-        assertThat(currentTsVer, equalTo(latestVer));
-    }
-
     private CompletableFuture<Void> changeColumn(
             String tab,
             String col,
diff --git a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/BaseIgniteAbstractTest.java b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/BaseIgniteAbstractTest.java
index 90e68ca971..8d3338cc9a 100644
--- a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/BaseIgniteAbstractTest.java
+++ b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/BaseIgniteAbstractTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.testframework;
 
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.lang.IgniteSystemProperties.IGNITE_SENSITIVE_DATA_LOGGING;
 import static org.apache.ignite.internal.lang.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.util.IgniteUtils.monotonicMs;
@@ -39,7 +38,6 @@ import org.mockito.Mockito;
  * Ignite base test class.
  */
 @ExtendWith(SystemPropertiesExtension.class)
-@WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "100")
 public abstract class BaseIgniteAbstractTest {
     /** Logger. */
     protected final IgniteLogger log = Loggers.forClass(getClass());
diff --git a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/TestIgnitionManager.java b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/TestIgnitionManager.java
index abc6cc085d..fd50c0550d 100644
--- a/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/TestIgnitionManager.java
+++ b/modules/core/src/testFixtures/java/org/apache/ignite/internal/testframework/TestIgnitionManager.java
@@ -37,10 +37,14 @@ public class TestIgnitionManager {
     /** Default name of configuration file. */
     public static final String DEFAULT_CONFIG_NAME = "ignite-config.conf";
 
-    private static final int DEFAULT_DELAY_DURATION_MS = 100;
+    /** Default DelayDuration in ms used for tests that is set on node init. */
+    public static final int DEFAULT_DELAY_DURATION_MS = 100;
 
     private static final int DEFAULT_METASTORAGE_IDLE_SYNC_TIME_INTERVAL_MS = 10;
 
+    /** Default partition idle SafeTime interval in ms used for tests that is set on node init. */
+    public static final int DEFAULT_PARTITION_IDLE_SYNC_TIME_INTERVAL_MS = 100;
+
     /**
      * Starts an Ignite node with an optional bootstrap configuration from an input stream with HOCON configs.
      *
@@ -122,6 +126,11 @@ public class TestIgnitionManager {
                 "metaStorage.idleSyncTimeInterval",
                 Integer.toString(DEFAULT_METASTORAGE_IDLE_SYNC_TIME_INTERVAL_MS)
         );
+        configDocument = applyTestDefault(
+                configDocument,
+                "replication.idleSafeTimePropagationDuration",
+                Integer.toString(DEFAULT_PARTITION_IDLE_SYNC_TIME_INTERVAL_MS)
+        );
 
         builder.clusterConfiguration(configDocument.render());
 
diff --git a/modules/replicator/build.gradle b/modules/replicator/build.gradle
index 8e40ee3a3b..aedb8de668 100644
--- a/modules/replicator/build.gradle
+++ b/modules/replicator/build.gradle
@@ -22,7 +22,10 @@ apply from: "$rootDir/buildscripts/java-integration-test.gradle"
 apply from: "$rootDir/buildscripts/java-test-fixtures.gradle"
 
 dependencies {
+    annotationProcessor project(':ignite-configuration-annotation-processor')
     annotationProcessor project(":ignite-network-annotation-processor")
+    annotationProcessor libs.auto.service
+
     implementation project(':ignite-core')
     implementation project(':ignite-raft')
     implementation project(':ignite-raft-api')
@@ -33,6 +36,7 @@ dependencies {
     implementation project(':ignite-placement-driver-api')
     implementation libs.jetbrains.annotations
     implementation libs.fastutil.core
+    implementation libs.auto.service.annotations
 
     integrationTestImplementation project(':ignite-raft')
     integrationTestImplementation project(':ignite-raft-api')
diff --git a/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
index 692b19ad5c..1879a8859b 100644
--- a/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
+++ b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.replicator;
 
 import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.stream.Collectors.toSet;
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.replicator.LocalReplicaEvent.AFTER_REPLICA_STARTED;
 import static org.apache.ignite.internal.replicator.LocalReplicaEvent.BEFORE_REPLICA_STOPPED;
 import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
@@ -758,16 +757,4 @@ public class ReplicaManager extends AbstractEventProducer<LocalReplicaEvent, Loc
     public Set<ReplicationGroupId> startedGroups() {
         return replicas.keySet();
     }
-
-    /**
-     * TODO: to be removed after IGNITE-20854 is fixed. This was introduced in a rush because of a burning release, should be fixe asap.
-     */
-    public static long idleSafeTimePropagationPeriodMs() {
-        return Long.parseLong(
-                System.getProperty(
-                        IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY,
-                        Integer.toString(DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS)
-                )
-        );
-    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/Kludges.java b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModule.java
similarity index 51%
copy from modules/core/src/main/java/org/apache/ignite/internal/Kludges.java
copy to modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModule.java
index de903ec28e..bb3bc0cc54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/Kludges.java
+++ b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModule.java
@@ -15,13 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal;
+package org.apache.ignite.internal.replicator.configuration;
+
+import com.google.auto.service.AutoService;
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.configuration.ConfigurationModule;
+import org.apache.ignite.configuration.RootKey;
+import org.apache.ignite.configuration.annotation.ConfigurationType;
 
 /**
- * Contains kludges needed for the whole codebase. Should be removed as quickly as possible.
+ * {@link ConfigurationModule} for distributed (cluster-wide) configuration of Replication.
  */
-public class Kludges {
-    // TODO: Remove after IGNITE-20854 is fixed.
-    /** Name of the property overriding idle safe time propagation period (in milliseconds). */
-    public static final String IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY = "IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS";
+@AutoService(ConfigurationModule.class)
+public class ReplicationConfigurationModule implements ConfigurationModule {
+    @Override
+    public ConfigurationType type() {
+        return ConfigurationType.DISTRIBUTED;
+    }
+
+    @Override
+    public Collection<RootKey<?, ?>> rootKeys() {
+        return Set.of(ReplicationConfiguration.KEY);
+    }
 }
diff --git a/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationSchema.java b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationSchema.java
new file mode 100644
index 0000000000..2a218e3c28
--- /dev/null
+++ b/modules/replicator/src/main/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationSchema.java
@@ -0,0 +1,36 @@
+/*
+ * 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.replicator.configuration;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.configuration.annotation.ConfigurationRoot;
+import org.apache.ignite.configuration.annotation.ConfigurationType;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Range;
+
+/**
+ * Configuration for Replication.
+ */
+@ConfigurationRoot(rootName = "replication", type = ConfigurationType.DISTRIBUTED)
+public class ReplicationConfigurationSchema {
+    /** Idle safe time propagation duration (ms) for partitions. */
+    @Value(hasDefault = true)
+    @Range(min = 0)
+    // TODO: IGNITE-19792 - make @Immutable when it gets being handled property for distributed config.
+    public long idleSafeTimePropagationDuration = TimeUnit.SECONDS.toMillis(1);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/Kludges.java b/modules/replicator/src/test/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModuleTest.java
similarity index 52%
rename from modules/core/src/main/java/org/apache/ignite/internal/Kludges.java
rename to modules/replicator/src/test/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModuleTest.java
index de903ec28e..763177cd17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/Kludges.java
+++ b/modules/replicator/src/test/java/org/apache/ignite/internal/replicator/configuration/ReplicationConfigurationModuleTest.java
@@ -15,13 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal;
+package org.apache.ignite.internal.replicator.configuration;
 
-/**
- * Contains kludges needed for the whole codebase. Should be removed as quickly as possible.
- */
-public class Kludges {
-    // TODO: Remove after IGNITE-20854 is fixed.
-    /** Name of the property overriding idle safe time propagation period (in milliseconds). */
-    public static final String IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY = "IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS";
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.is;
+
+import org.apache.ignite.configuration.ConfigurationModule;
+import org.apache.ignite.configuration.annotation.ConfigurationType;
+import org.junit.jupiter.api.Test;
+
+class ReplicationConfigurationModuleTest {
+    private final ConfigurationModule module = new ReplicationConfigurationModule();
+
+    @Test
+    void typeIsDistributed() {
+        assertThat(module.type(), is(ConfigurationType.DISTRIBUTED));
+    }
+
+    @Test
+    void rootKeysAreAsExpected() {
+        assertThat(module.rootKeys(), contains(ReplicationConfiguration.KEY));
+    }
 }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/readonly/ItReadOnlyTxInPastTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/readonly/ItReadOnlyTxInPastTest.java
index fdc00052c3..ad59fc551a 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/readonly/ItReadOnlyTxInPastTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/readonly/ItReadOnlyTxInPastTest.java
@@ -17,14 +17,12 @@
 
 package org.apache.ignite.internal.readonly;
 
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.SessionUtils.executeUpdate;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
 
 import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
 import org.apache.ignite.internal.app.IgniteImpl;
-import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.apache.ignite.sql.ResultSet;
 import org.apache.ignite.sql.SqlRow;
 import org.apache.ignite.tx.TransactionOptions;
@@ -35,8 +33,6 @@ import org.junit.jupiter.api.Test;
  * Tests about read-only transactions in the past.
  */
 @SuppressWarnings("resource")
-// Setting this to 1 second so that an RO tx has a potential to look before a table was created.
-@WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "1000")
 class ItReadOnlyTxInPastTest extends ClusterPerTestIntegrationTest {
     private static final String TABLE_NAME = "test";
 
@@ -47,7 +43,8 @@ class ItReadOnlyTxInPastTest extends ClusterPerTestIntegrationTest {
 
     @BeforeEach
     void prepareCluster() {
-        cluster.startAndInit(1);
+        // Setting idleSafeTimePropagationDuration to 1 second so that an RO tx has a potential to look before a table was created.
+        cluster.startAndInit(1, builder -> builder.clusterConfiguration("replication.idleSafeTimePropagationDuration: 1000"));
 
         cluster.doInSession(0, session -> {
             executeUpdate("CREATE TABLE " + TABLE_NAME + " (id int PRIMARY KEY, val varchar)", session);
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
index a140ba43d0..c5a535c5a0 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.runner.app;
 
 import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.concurrent.CompletableFuture.failedFuture;
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
@@ -125,7 +124,6 @@ import org.apache.ignite.internal.table.distributed.raft.snapshot.outgoing.Outgo
 import org.apache.ignite.internal.table.distributed.schema.SchemaSyncServiceImpl;
 import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
 import org.apache.ignite.internal.testframework.TestIgnitionManager;
-import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.apache.ignite.internal.tx.HybridTimestampTracker;
 import org.apache.ignite.internal.tx.impl.HeapLockManager;
 import org.apache.ignite.internal.tx.impl.TransactionIdGenerator;
@@ -158,7 +156,6 @@ import org.junit.jupiter.params.provider.ValueSource;
  */
 @ExtendWith(ConfigurationExtension.class)
 @Timeout(120)
-@WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "200")
 public class ItIgniteNodeRestartTest extends BaseIgniteRestartTest {
     /** Value producer for table data, is used to create data and check it later. */
     private static final IntFunction<String> VALUE_PRODUCER = i -> "val " + i;
@@ -280,7 +277,8 @@ public class ItIgniteNodeRestartTest extends BaseIgniteRestartTest {
                 new NodeAttributesCollector(nodeAttributes, storageProfilesConfiguration)
         );
 
-        LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier = () -> 10L;
+        LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier
+                = () -> TestIgnitionManager.DEFAULT_PARTITION_IDLE_SYNC_TIME_INTERVAL_MS;
 
         ReplicaManager replicaMgr = new ReplicaManager(
                 name,
@@ -348,7 +346,6 @@ public class ItIgniteNodeRestartTest extends BaseIgniteRestartTest {
 
         Consumer<LongFunction<CompletableFuture<?>>> registry = (c) -> metaStorageMgr.registerRevisionUpdateListener(c::apply);
 
-
         DataStorageModules dataStorageModules = new DataStorageModules(ServiceLoader.load(DataStorageModule.class));
 
         Path storagePath = getPartitionsStorePath(dir);
@@ -366,7 +363,7 @@ public class ItIgniteNodeRestartTest extends BaseIgniteRestartTest {
 
         var clockWaiter = new ClockWaiter(name, hybridClock);
 
-        LongSupplier delayDurationMsSupplier = () -> 100L;
+        LongSupplier delayDurationMsSupplier = () -> TestIgnitionManager.DEFAULT_DELAY_DURATION_MS;
 
         var catalogManager = new CatalogManagerImpl(
                 new UpdateLogImpl(metaStorageMgr),
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 a7f6d4c006..8c6d88a75b 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
@@ -115,6 +115,7 @@ import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
 import org.apache.ignite.internal.raft.storage.impl.VolatileLogStorageFactoryCreator;
 import org.apache.ignite.internal.replicator.ReplicaManager;
 import org.apache.ignite.internal.replicator.ReplicaService;
+import org.apache.ignite.internal.replicator.configuration.ReplicationConfiguration;
 import org.apache.ignite.internal.rest.RestComponent;
 import org.apache.ignite.internal.rest.RestFactory;
 import org.apache.ignite.internal.rest.authentication.AuthenticationProviderFactory;
@@ -484,7 +485,9 @@ public class IgniteImpl implements Ignite {
                 clock
         );
 
-        LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier = partitionIdleSafeTimePropagationPeriodMsSupplier();
+        ReplicationConfiguration replicationConfig = clusterConfigRegistry.getConfiguration(ReplicationConfiguration.KEY);
+
+        LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier = partitionIdleSafeTimePropagationPeriodMsSupplier(replicationConfig);
 
         replicaMgr = new ReplicaManager(
                 name,
@@ -526,7 +529,7 @@ public class IgniteImpl implements Ignite {
                 SchemaSynchronizationConfiguration.KEY
         );
 
-        LongSupplier delayDurationMsSupplier = () -> schemaSyncConfig.delayDuration().value();
+        LongSupplier delayDurationMsSupplier = delayDurationMsSupplier(schemaSyncConfig);
 
         CatalogManagerImpl catalogManager = new CatalogManagerImpl(
                 new UpdateLogImpl(metaStorageMgr),
@@ -670,9 +673,12 @@ public class IgniteImpl implements Ignite {
         restComponent = createRestComponent(name);
     }
 
-    private static LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier() {
-        // TODO: Replace with an immutable dynamic property set on cluster init after IGNITE-20854 is fixed.
-        return ReplicaManager::idleSafeTimePropagationPeriodMs;
+    private static SameValueLongSupplier delayDurationMsSupplier(SchemaSynchronizationConfiguration schemaSyncConfig) {
+        return new SameValueLongSupplier(() -> schemaSyncConfig.delayDuration().value());
+    }
+
+    private static LongSupplier partitionIdleSafeTimePropagationPeriodMsSupplier(ReplicationConfiguration replicationConfig) {
+        return new SameValueLongSupplier(() -> replicationConfig.idleSafeTimePropagationDuration().value());
     }
 
     private AuthenticationManager createAuthenticationManager() {
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/SameValueLongSupplier.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/SameValueLongSupplier.java
new file mode 100644
index 0000000000..ade1adaa4f
--- /dev/null
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/SameValueLongSupplier.java
@@ -0,0 +1,49 @@
+/*
+ * 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.app;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongSupplier;
+import java.util.function.LongUnaryOperator;
+
+/**
+ * {@link LongSupplier} that fails an assertion if a wrapped supplier returns a value different
+ * from the one returned on the previous call.
+ */
+class SameValueLongSupplier implements LongSupplier {
+    private static final long NO_VALUE = Long.MIN_VALUE;
+
+    private final AtomicLong previousValue = new AtomicLong(NO_VALUE);
+
+    private final LongUnaryOperator assertingUpdate;
+
+    SameValueLongSupplier(LongSupplier supplier) {
+        assertingUpdate = prev -> {
+            long current = supplier.getAsLong();
+
+            assert prev == NO_VALUE || current == prev : "Previous value was " + prev + ", but current value is " + current;
+
+            return current;
+        };
+    }
+
+    @Override
+    public long getAsLong() {
+        return previousValue.updateAndGet(assertingUpdate);
+    }
+}
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/app/SameValueLongSupplierTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/app/SameValueLongSupplierTest.java
new file mode 100644
index 0000000000..ef3e44086f
--- /dev/null
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/app/SameValueLongSupplierTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.app;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.when;
+
+import java.util.function.LongSupplier;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class SameValueLongSupplierTest extends BaseIgniteAbstractTest {
+    @Mock
+    private LongSupplier supplier;
+
+    @InjectMocks
+    private SameValueLongSupplier sameValueSupplier;
+
+    @Test
+    void suppliesSameValueSuccessfully() {
+        when(supplier.getAsLong()).thenReturn(1L, 1L, 1L);
+
+        assertThat(sameValueSupplier.getAsLong(), is(1L));
+        assertThat(sameValueSupplier.getAsLong(), is(1L));
+        assertThat(sameValueSupplier.getAsLong(), is(1L));
+    }
+
+    @Test
+    void failsAssertionOnMismatchingValue() {
+        when(supplier.getAsLong()).thenReturn(2L, 3L);
+
+        sameValueSupplier.getAsLong();
+
+        AssertionError error = assertThrows(AssertionError.class, () -> sameValueSupplier.getAsLong());
+        assertThat(error.getMessage(), is("Previous value was 2, but current value is 3"));
+    }
+}
diff --git a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
index bf8032fe28..f9612cef29 100644
--- a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
+++ b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.nio.file.Path;
 import java.util.List;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.InitParametersBuilder;
 import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
@@ -102,7 +103,7 @@ public abstract class ClusterPerTestIntegrationTest extends IgniteIntegrationTes
         cluster = new Cluster(testInfo, workDir, getNodeBootstrapConfigTemplate());
 
         if (initialNodes() > 0) {
-            cluster.startAndInit(initialNodes(), cmgMetastoreNodes());
+            cluster.startAndInit(initialNodes(), cmgMetastoreNodes(), this::customizeInitParameters);
         }
     }
 
@@ -125,6 +126,10 @@ public abstract class ClusterPerTestIntegrationTest extends IgniteIntegrationTes
         return new int[] { 0 };
     }
 
+    protected void customizeInitParameters(InitParametersBuilder builder) {
+        // No-op.
+    }
+
     /**
      * Returns node bootstrap config template.
      *
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
index 1a6d316c71..a13ac93463 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.rebalance;
 
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -27,13 +26,11 @@ import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
 import org.apache.ignite.internal.storage.MvPartitionStorage;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.test.WatchListenerInhibitor;
-import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.junit.jupiter.api.Test;
 
 /**
  * Tests for recovery of the rebalance procedure.
  */
-@WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "1000")
 public class ItRebalanceRecoveryTest extends ClusterPerTestIntegrationTest {
     @Override
     protected int initialNodes() {
diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
index 5db62701c0..1b2dde012a 100644
--- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
+++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
@@ -19,10 +19,9 @@ package org.apache.ignite.internal.tx;
 
 
 import static java.lang.Math.abs;
-import static org.apache.ignite.internal.Kludges.IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY;
 import static org.apache.ignite.internal.hlc.HybridTimestamp.CLOCK_SKEW;
 import static org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestamp;
-import static org.apache.ignite.internal.replicator.ReplicaManager.idleSafeTimePropagationPeriodMs;
+import static org.apache.ignite.internal.replicator.ReplicaManager.DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
 import static org.apache.ignite.lang.ErrorGroups.Transactions.TX_COMMIT_ERR;
@@ -44,6 +43,7 @@ import static org.mockito.Mockito.when;
 
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
+import java.util.function.LongSupplier;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridClockImpl;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.placementdriver.TestReplicaMetaImpl;
 import org.apache.ignite.internal.replicator.ReplicaService;
 import org.apache.ignite.internal.replicator.TablePartitionId;
 import org.apache.ignite.internal.testframework.IgniteAbstractTest;
-import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.apache.ignite.internal.tx.impl.HeapLockManager;
 import org.apache.ignite.internal.tx.impl.PrimaryReplicaExpiredException;
 import org.apache.ignite.internal.tx.impl.TransactionIdGenerator;
@@ -86,6 +85,8 @@ public class TxManagerTest extends IgniteAbstractTest {
     /** Timestamp tracker. */
     private HybridTimestampTracker hybridTimestampTracker = new HybridTimestampTracker();
 
+    private final LongSupplier idleSafeTimePropagationPeriodMsSupplier = () -> DEFAULT_IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS;
+
     private TxManager txManager;
 
     @Mock
@@ -118,7 +119,8 @@ public class TxManagerTest extends IgniteAbstractTest {
                 clock,
                 new TransactionIdGenerator(0xdeadbeef),
                 LOCAL_NODE::id,
-                placementDriver
+                placementDriver,
+                idleSafeTimePropagationPeriodMsSupplier
         );
 
         txManager.start();
@@ -276,11 +278,10 @@ public class TxManagerTest extends IgniteAbstractTest {
     }
 
     @Test
-    @WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "1000")
     public void testObservableTimestamp() {
         long compareThreshold = 50;
         // Check that idle safe time propagation period is significantly greater than compareThreshold.
-        assertTrue(idleSafeTimePropagationPeriodMs() + CLOCK_SKEW > compareThreshold * 5);
+        assertTrue(idleSafeTimePropagationPeriodMsSupplier.getAsLong() + CLOCK_SKEW > compareThreshold * 5);
 
         HybridTimestamp now = clock.now();
 
@@ -297,7 +298,7 @@ public class TxManagerTest extends IgniteAbstractTest {
         tx.commit();
 
         HybridTimestamp timestampInPast = new HybridTimestamp(
-                now.getPhysical() - idleSafeTimePropagationPeriodMs() * 2,
+                now.getPhysical() - idleSafeTimePropagationPeriodMsSupplier.getAsLong() * 2,
                 now.getLogical()
         );
 
@@ -307,7 +308,7 @@ public class TxManagerTest extends IgniteAbstractTest {
 
         tx = txManager.begin(hybridTimestampTracker, true);
 
-        long readTime = now.getPhysical() - idleSafeTimePropagationPeriodMs() - CLOCK_SKEW;
+        long readTime = now.getPhysical() - idleSafeTimePropagationPeriodMsSupplier.getAsLong() - CLOCK_SKEW;
 
         assertThat(abs(readTime - tx.readTimestamp().getPhysical()), Matchers.lessThan(compareThreshold));
 
@@ -315,11 +316,10 @@ public class TxManagerTest extends IgniteAbstractTest {
     }
 
     @Test
-    @WithSystemProperty(key = IDLE_SAFE_TIME_PROPAGATION_PERIOD_MILLISECONDS_PROPERTY, value = "1000")
     public void testObservableTimestampLocally() {
         long compareThreshold = 50;
         // Check that idle safe time propagation period is significantly greater than compareThreshold.
-        assertTrue(idleSafeTimePropagationPeriodMs() + CLOCK_SKEW > compareThreshold * 5);
+        assertTrue(idleSafeTimePropagationPeriodMsSupplier.getAsLong() + CLOCK_SKEW > compareThreshold * 5);
 
         HybridTimestamp now = clock.now();
 
@@ -330,7 +330,7 @@ public class TxManagerTest extends IgniteAbstractTest {
         assertTrue(firstReadTs.compareTo(now) < 0);
 
         assertTrue(now.getPhysical() - firstReadTs.getPhysical() < compareThreshold
-                + idleSafeTimePropagationPeriodMs() + CLOCK_SKEW);
+                + idleSafeTimePropagationPeriodMsSupplier.getAsLong() + CLOCK_SKEW);
         tx.commit();
 
         tx = txManager.begin(hybridTimestampTracker, true);
@@ -338,7 +338,7 @@ public class TxManagerTest extends IgniteAbstractTest {
         assertTrue(firstReadTs.compareTo(tx.readTimestamp()) <= 0);
 
         assertTrue(abs(now.getPhysical() - tx.readTimestamp().getPhysical()) < compareThreshold
-                + idleSafeTimePropagationPeriodMs() + CLOCK_SKEW);
+                + idleSafeTimePropagationPeriodMsSupplier.getAsLong() + CLOCK_SKEW);
         tx.commit();
     }