You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2019/06/05 11:08:26 UTC

[ignite] 23/31: GG-18917 [IGNITE-11821] Deprecate rebalance properties at CacheConfiguration and move them to IgniteConfiguration level (#6534)

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

amashenkov pushed a commit to branch gg-19225
in repository https://gitbox.apache.org/repos/asf/ignite.git

commit 81b8d8519dcdeb23b3e9c6c2f552cb4a4aafd892
Author: Maxim Muzafarov <ma...@gmail.com>
AuthorDate: Mon Jun 3 16:59:22 2019 +0300

    GG-18917 [IGNITE-11821] Deprecate rebalance properties at CacheConfiguration and move them to IgniteConfiguration level (#6534)
    
    (cherry picked from commit 64c6686da2d9c7fd730147089dc50ac8bc4b78c9)
---
 .../org/apache/ignite/IgniteSystemProperties.java  |  11 +-
 .../ignite/client/ClientCacheConfiguration.java    |   9 +-
 .../ignite/configuration/CacheConfiguration.java   |  63 +++++++--
 .../ignite/configuration/IgniteConfiguration.java  | 155 +++++++++++++++++++++
 .../org/apache/ignite/internal/IgniteKernal.java   |  16 +++
 .../processors/cache/GridCacheAttributes.java      |   9 ++
 .../processors/cache/GridCachePreloader.java       |  45 ++++++
 .../cache/GridCachePreloaderAdapter.java           |  29 ++++
 .../processors/cache/GridCacheProcessor.java       |   6 +-
 .../dht/preloader/GridDhtPartitionDemander.java    |  10 +-
 .../dht/preloader/GridDhtPartitionSupplier.java    |  24 ++--
 .../GridCacheAbstractNodeRestartSelfTest.java      |   2 +-
 .../GridCachePreloadRestartAbstractSelfTest.java   |   2 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java    |   2 +-
 .../dht/GridCacheDhtPreloadSelfTest.java           |   2 +-
 .../dht/GridCacheDhtPreloadStartStopSelfTest.java  |   2 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java     |   2 +-
 .../configvariations/ConfigVariations.java         |   2 +-
 .../ApiParity/IgniteConfigurationParityTest.cs     |   4 +
 19 files changed, 352 insertions(+), 43 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 904efe7..4e7b312 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1,12 +1,12 @@
 /*
  * Copyright 2019 GridGain Systems, Inc. and Contributors.
- * 
+ *
  * Licensed under the GridGain Community Edition License (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at
- * 
+ *
  *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
- * 
+ *
  * 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.
@@ -28,6 +28,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CheckpointWriteOrder;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
@@ -1021,9 +1022,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_WAIT_SCHEMA_UPDATE = "IGNITE_WAIT_SCHEMA_UPDATE";
 
     /**
-     * System property to override {@link CacheConfiguration#rebalanceThrottle} configuration property for all caches.
+     * System property to override {@link CacheConfiguration#getRebalanceThrottle} configuration property for all caches.
      * {@code 0} by default, which means that override is disabled.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceThrottle()} instead.
      */
+    @Deprecated
     public static final String IGNITE_REBALANCE_THROTTLE_OVERRIDE = "IGNITE_REBALANCE_THROTTLE_OVERRIDE";
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/client/ClientCacheConfiguration.java
index 2777c9c..cf40ead 100644
--- a/modules/core/src/main/java/org/apache/ignite/client/ClientCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientCacheConfiguration.java
@@ -25,6 +25,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /** Cache configuration. */
@@ -60,10 +61,10 @@ public final class ClientCacheConfiguration implements Serializable {
     private boolean readFromBackup = CacheConfiguration.DFLT_READ_FROM_BACKUP;
 
     /** @serial Rebalance batch size. */
-    private int rebalanceBatchSize = CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
+    private int rebalanceBatchSize = IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 
     /** @serial Rebalance batches prefetch count. */
-    private long rebalanceBatchesPrefetchCnt = CacheConfiguration.DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
+    private long rebalanceBatchesPrefetchCnt = IgniteConfiguration.DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
 
     /** @serial Rebalance delay. */
     private long rebalanceDelay = 0;
@@ -75,10 +76,10 @@ public final class ClientCacheConfiguration implements Serializable {
     private int rebalanceOrder = 0;
 
     /** @serial Rebalance throttle. */
-    private long rebalanceThrottle = CacheConfiguration.DFLT_REBALANCE_THROTTLE;
+    private long rebalanceThrottle = IgniteConfiguration.DFLT_REBALANCE_THROTTLE;
 
     /** @serial @serial Rebalance timeout. */
-    private long rebalanceTimeout = CacheConfiguration.DFLT_REBALANCE_TIMEOUT;
+    private long rebalanceTimeout = IgniteConfiguration.DFLT_REBALANCE_TIMEOUT;
 
     /** @serial Write synchronization mode. */
     private CacheWriteSynchronizationMode writeSynchronizationMode = CacheWriteSynchronizationMode.PRIMARY_SYNC;
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 66d951b..c7208f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -80,18 +80,33 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Maximum number of partitions. */
     public static final int MAX_PARTITIONS_COUNT = 65000;
 
-    /** Default size of rebalance thread pool. */
+    /**
+     * Default size of rebalance thread pool.
+     * @deprecated Use {@link IgniteConfiguration#DFLT_REBALANCE_THREAD_POOL_SIZE} instead.
+     */
     @Deprecated
-    public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = 2;
+    public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = IgniteConfiguration.DFLT_REBALANCE_THREAD_POOL_SIZE;
 
-    /** Default rebalance timeout (ms).*/
-    public static final long DFLT_REBALANCE_TIMEOUT = 10000;
+    /**
+     * Default rebalance timeout (ms).
+     * @deprecated Use {@link IgniteConfiguration#DFLT_REBALANCE_TIMEOUT} instead.
+     */
+    @Deprecated
+    public static final long DFLT_REBALANCE_TIMEOUT = IgniteConfiguration.DFLT_REBALANCE_TIMEOUT;
 
-    /** Default rebalance batches prefetch count. */
-    public static final long DFLT_REBALANCE_BATCHES_PREFETCH_COUNT = 2;
+    /**
+     * Default rebalance batches prefetch count.
+     * @deprecated Use {@link IgniteConfiguration#DFLT_REBALANCE_BATCHES_PREFETCH_COUNT} instead.
+     */
+    @Deprecated
+    public static final long DFLT_REBALANCE_BATCHES_PREFETCH_COUNT = IgniteConfiguration.DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
 
-    /** Time in milliseconds to wait between rebalance messages to avoid overloading CPU. */
-    public static final long DFLT_REBALANCE_THROTTLE = 0;
+    /**
+     * Time in milliseconds to wait between rebalance messages to avoid overloading CPU.
+     * @deprecated Use {@link IgniteConfiguration#DFLT_REBALANCE_THROTTLE} instead.
+     */
+    @Deprecated
+    public static final long DFLT_REBALANCE_THROTTLE = IgniteConfiguration.DFLT_REBALANCE_THROTTLE;
 
     /** Default number of backups. */
     public static final int DFLT_BACKUPS = 0;
@@ -120,8 +135,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default rebalance mode for distributed cache. */
     public static final CacheRebalanceMode DFLT_REBALANCE_MODE = CacheRebalanceMode.ASYNC;
 
-    /** Default rebalance batch size in bytes. */
-    public static final int DFLT_REBALANCE_BATCH_SIZE = 512 * 1024; // 512K
+    /**
+     * Default rebalance batch size in bytes.
+     * @deprecated Use {@link IgniteConfiguration#DFLT_REBALANCE_BATCH_SIZE} instead.
+     */
+    @Deprecated
+    public static final int DFLT_REBALANCE_BATCH_SIZE = IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 
     /** Default value for eager ttl flag. */
     public static final boolean DFLT_EAGER_TTL = true;
@@ -205,6 +224,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private int rebalancePoolSize = DFLT_REBALANCE_THREAD_POOL_SIZE;
 
     /** Rebalance timeout. */
+    @Deprecated
     private long rebalanceTimeout = DFLT_REBALANCE_TIMEOUT;
 
     /** Cache eviction policy. */
@@ -278,9 +298,11 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private int rebalanceOrder;
 
     /** Rebalance batch size. */
+    @Deprecated
     private int rebalanceBatchSize = DFLT_REBALANCE_BATCH_SIZE;
 
     /** Rebalance batches prefetch count. */
+    @Deprecated
     private long rebalanceBatchesPrefetchCnt = DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
 
     /** Maximum number of concurrent asynchronous operations. */
@@ -316,7 +338,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** */
     private long rebalanceDelay;
 
-    /** */
+    /** Time in milliseconds to wait between rebalance messages to avoid overloading CPU. */
+    @Deprecated
     private long rebalanceThrottle = DFLT_REBALANCE_THROTTLE;
 
     /** */
@@ -1200,7 +1223,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * {@link #DFLT_REBALANCE_BATCH_SIZE}.
      *
      * @return Size in bytes of a single rebalance message.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceBatchSize()} instead.
      */
+    @Deprecated
     public int getRebalanceBatchSize() {
         return rebalanceBatchSize;
     }
@@ -1210,7 +1235,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param rebalanceBatchSize Rebalance batch size.
      * @return {@code this} for chaining.
+     * @deprecated Use {@link IgniteConfiguration#setRebalanceBatchSize(int)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setRebalanceBatchSize(int rebalanceBatchSize) {
         this.rebalanceBatchSize = rebalanceBatchSize;
 
@@ -1225,7 +1252,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Minimum is 1.
      *
      * @return batches count
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceBatchesPrefetchCount()} instead.
      */
+    @Deprecated
     public long getRebalanceBatchesPrefetchCount() {
         return rebalanceBatchesPrefetchCnt;
     }
@@ -1239,7 +1268,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param rebalanceBatchesCnt batches count.
      * @return {@code this} for chaining.
+     * @deprecated Use {@link IgniteConfiguration#setRebalanceBatchesPrefetchCount(long)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setRebalanceBatchesPrefetchCount(long rebalanceBatchesCnt) {
         this.rebalanceBatchesPrefetchCnt = rebalanceBatchesCnt;
 
@@ -1488,7 +1519,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Default value is {@link #DFLT_REBALANCE_TIMEOUT}.
      *
      * @return Rebalance timeout (ms).
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceTimeout()} instead.
      */
+    @Deprecated
     public long getRebalanceTimeout() {
         return rebalanceTimeout;
     }
@@ -1498,7 +1531,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param rebalanceTimeout Rebalance timeout (ms).
      * @return {@code this} for chaining.
+     * @deprecated Use {@link IgniteConfiguration#setRebalanceTimeout(long)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setRebalanceTimeout(long rebalanceTimeout) {
         this.rebalanceTimeout = rebalanceTimeout;
 
@@ -1555,8 +1590,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * the default is defined by {@link #DFLT_REBALANCE_THROTTLE} constant.
      *
      * @return Time in milliseconds to wait between rebalance messages to avoid overloading of CPU,
-     *      {@code 0} to disable throttling.
+     * {@code 0} to disable throttling.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceThrottle()} instead.
      */
+    @Deprecated
     public long getRebalanceThrottle() {
         return rebalanceThrottle;
     }
@@ -1572,7 +1609,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalanceThrottle Time in milliseconds to wait between rebalance messages to avoid overloading of CPU,
      * {@code 0} to disable throttling.
      * @return {@code this} for chaining.
+     * @deprecated Use {@link IgniteConfiguration#setRebalanceThrottle(long)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setRebalanceThrottle(long rebalanceThrottle) {
         this.rebalanceThrottle = rebalanceThrottle;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 4062b92..c35d22f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -155,6 +155,18 @@ public class IgniteConfiguration {
     /** Default limit of threads used for rebalance. */
     public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = 1;
 
+    /** Default rebalance message timeout in milliseconds (value is {@code 10000}). */
+    public static final long DFLT_REBALANCE_TIMEOUT = 10000;
+
+    /** Default rebalance batches prefetch count (value is {@code 2}). */
+    public static final long DFLT_REBALANCE_BATCHES_PREFETCH_COUNT = 2;
+
+    /** Time to wait between rebalance messages in milliseconds to avoid overloading CPU (value is {@code 0}). */
+    public static final long DFLT_REBALANCE_THROTTLE = 0;
+
+    /** Default rebalance batch size in bytes (value is {@code 512Kb}). */
+    public static final int DFLT_REBALANCE_BATCH_SIZE = 512 * 1024; // 512K
+
     /** Default size of system thread pool. */
     public static final int DFLT_SYSTEM_CORE_THREAD_CNT = DFLT_PUBLIC_THREAD_CNT;
 
@@ -386,6 +398,18 @@ public class IgniteConfiguration {
     /** Rebalance thread pool size. */
     private int rebalanceThreadPoolSize = DFLT_REBALANCE_THREAD_POOL_SIZE;
 
+    /** Rrebalance messages timeout in milliseconds. */
+    private long rebalanceTimeout = DFLT_REBALANCE_TIMEOUT;
+
+    /** Rebalance batches prefetch count. */
+    private long rebalanceBatchesPrefetchCnt = DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
+
+    /** Time to wait between rebalance messages in milliseconds. */
+    private long rebalanceThrottle = DFLT_REBALANCE_THROTTLE;
+
+    /** Rebalance batch size in bytes. */
+    private int rebalanceBatchSize = DFLT_REBALANCE_BATCH_SIZE;
+
     /** Transactions configuration. */
     private TransactionConfiguration txCfg = new TransactionConfiguration();
 
@@ -619,6 +643,10 @@ public class IgniteConfiguration {
         pubPoolSize = cfg.getPublicThreadPoolSize();
         qryPoolSize = cfg.getQueryThreadPoolSize();
         rebalanceThreadPoolSize = cfg.getRebalanceThreadPoolSize();
+        rebalanceTimeout = cfg.getRebalanceTimeout();
+        rebalanceBatchesPrefetchCnt = cfg.getRebalanceBatchesPrefetchCount();
+        rebalanceThrottle = cfg.getRebalanceThrottle();
+        rebalanceBatchSize = cfg.getRebalanceBatchSize();
         segChkFreq = cfg.getSegmentCheckFrequency();
         segPlc = cfg.getSegmentationPolicy();
         segResolveAttempts = cfg.getSegmentationResolveAttempts();
@@ -1590,6 +1618,133 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Rebalance timeout for supply and demand messages in milliseconds. The {@code rebalanceTimeout} parameter
+     * specifies how long a message will stay in a receiving queue, waiting for other ordered messages that are
+     * ordered ahead of it to arrive will be processed. If timeout expires, then all messages that have not arrived
+     * before this message will be skipped. If an expired supply (demand) message actually does arrive, it will be
+     * ignored.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_TIMEOUT}, if {@code 0} than the
+     * {@link IgniteConfiguration#getNetworkTimeout()} will be used instead.
+     *
+     * @return Rebalance message timeout in milliseconds.
+     */
+    public long getRebalanceTimeout() {
+        return rebalanceTimeout;
+    }
+
+    /**
+     * Rebalance timeout for supply and demand messages in milliseconds. The {@code rebalanceTimeout} parameter
+     * specifies how long a message will stay in a receiving queue, waiting for other ordered messages that are
+     * ordered ahead of it to arrive will be processed. If timeout expires, then all messages that have not arrived
+     * before this message will be skipped. If an expired supply (demand) message actually does arrive, it will be
+     * ignored.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_TIMEOUT}, if {@code 0} than the
+     * {@link IgniteConfiguration#getNetworkTimeout()} will be used instead.
+     *
+     * @param rebalanceTimeout Rebalance message timeout in milliseconds.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setRebalanceTimeout(long rebalanceTimeout) {
+        this.rebalanceTimeout = rebalanceTimeout;
+
+        return this;
+    }
+
+    /**
+     * The number of batches generated by supply node at rebalancing procedure start. To gain better rebalancing
+     * performance supplier node can provide more than one batch at rebalancing start and provide one new to each
+     * next demand request.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_BATCHES_PREFETCH_COUNT}, minimum value is {@code 1}.
+     *
+     * @return The number of batches prefetch count.
+     */
+    public long getRebalanceBatchesPrefetchCount() {
+        return rebalanceBatchesPrefetchCnt;
+    }
+
+    /**
+     * The number of batches generated by supply node at rebalancing procedure start. To gain better rebalancing
+     * performance supplier node can provide more than one batch at rebalancing start and provide one new to each
+     * next demand request.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_BATCHES_PREFETCH_COUNT}, minimum value is {@code 1}.
+     *
+     * @param rebalanceBatchesCnt The number of batches prefetch count.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setRebalanceBatchesPrefetchCount(long rebalanceBatchesCnt) {
+        this.rebalanceBatchesPrefetchCnt = rebalanceBatchesCnt;
+
+        return this;
+    }
+
+    /**
+     * Time in milliseconds to wait between rebalance messages to avoid overloading of CPU or network.
+     * When rebalancing large data sets, the CPU or network can get over-consumed with rebalancing messages,
+     * which consecutively may slow down the application performance. This parameter helps tune
+     * the amount of time to wait between rebalance messages to make sure that rebalancing process
+     * does not have any negative performance impact. Note that application will continue to work
+     * properly while rebalancing is still in progress.
+     * <p>
+     * Value of {@code 0} means that throttling is disabled. By default throttling is disabled -
+     * the default is defined by {@link IgniteConfiguration#DFLT_REBALANCE_THROTTLE} constant.
+     *
+     * @return Time in milliseconds to wait between rebalance messages, {@code 0} to disable throttling.
+     */
+    public long getRebalanceThrottle() {
+        return rebalanceThrottle;
+    }
+
+    /**
+     * Time in milliseconds to wait between rebalance messages to avoid overloading of CPU or network. When rebalancing
+     * large data sets, the CPU or network can get over-consumed with rebalancing messages, which consecutively may slow
+     * down the application performance. This parameter helps tune the amount of time to wait between rebalance messages
+     * to make sure that rebalancing process does not have any negative performance impact. Note that application will
+     * continue to work properly while rebalancing is still in progress.
+     * <p>
+     * Value of {@code 0} means that throttling is disabled. By default throttling is disabled -
+     * the default is defined by {@link IgniteConfiguration#DFLT_REBALANCE_THROTTLE} constant.
+     *
+     * @param rebalanceThrottle Time in milliseconds to wait between rebalance messages, {@code 0} to disable throttling.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setRebalanceThrottle(long rebalanceThrottle) {
+        this.rebalanceThrottle = rebalanceThrottle;
+
+        return this;
+    }
+
+    /**
+     * The supply message size in bytes to be loaded within a single rebalance batch. The data balancing algorithm
+     * splits all the cache data entries on supply node into multiple batches prior to sending them to the demand node.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_BATCH_SIZE}.
+     *
+     * @return Rebalance message size in bytes.
+     */
+    public int getRebalanceBatchSize() {
+        return rebalanceBatchSize;
+    }
+
+    /**
+     * The supply message size in bytes to be loaded within a single rebalance batch. The data balancing algorithm
+     * splits all the cache data entries on supply node into multiple batches prior to sending them to the demand node.
+     * <p>
+     * Default value is defined by {@link IgniteConfiguration#DFLT_REBALANCE_BATCH_SIZE}.
+     *
+     * @param rebalanceBatchSize Rebalance message size in bytes.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setRebalanceBatchSize(int rebalanceBatchSize) {
+        this.rebalanceBatchSize = rebalanceBatchSize;
+
+        return this;
+    }
+
+    /**
      * Returns a collection of life-cycle beans. These beans will be automatically
      * notified of grid life-cycle events. Use life-cycle beans whenever you
      * want to perform certain logic before and after grid startup and stopping
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index c3dbf07..f324d7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2597,6 +2597,22 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 throw new IgniteCheckedException("Rebalance thread pool size minimal allowed value is 1. " +
                     "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
 
+            if (cfg.getRebalanceBatchesPrefetchCount() < 1)
+                throw new IgniteCheckedException("Rebalance batches prefetch count minimal allowed value is 1. " +
+                    "Change IgniteConfiguration.rebalanceBatchesPrefetchCount property before next start.");
+
+            if (cfg.getRebalanceBatchSize() <= 0)
+                throw new IgniteCheckedException("Rebalance batch size must be greater than zero. " +
+                    "Change IgniteConfiguration.rebalanceBatchSize property before next start.");
+
+            if (cfg.getRebalanceThrottle() < 0)
+                throw new IgniteCheckedException("Rebalance throttle can't have negative value. " +
+                    "Change IgniteConfiguration.rebalanceThrottle property before next start.");
+
+            if (cfg.getRebalanceTimeout() < 0)
+                throw new IgniteCheckedException("Rebalance message timeout can't have negative value. " +
+                    "Change IgniteConfiguration.rebalanceTimeout property before next start.");
+
             for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) {
                 if (ccfg.getRebalanceBatchesPrefetchCount() < 1)
                     throw new IgniteCheckedException("Rebalance batches prefetch count minimal allowed value is 1. " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 371d8e9..a51c7bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -250,7 +251,9 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @return Preload batch size.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceBatchSize()} instead.
      */
+    @Deprecated
     public int rebalanceBatchSize() {
         return ccfg.getRebalanceBatchSize();
     }
@@ -264,7 +267,9 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @return Rebalance prefetch count.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceBatchesPrefetchCount()} instead.
      */
+    @Deprecated
     public long rebalanceBatchesPrefetchCount() {
         return ccfg.getRebalanceBatchesPrefetchCount();
     }
@@ -278,14 +283,18 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @return Rebalance throttle.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceThrottle()} instead.
      */
+    @Deprecated
     public long rebalanceThrottle() {
         return ccfg.getRebalanceThrottle();
     }
 
     /**
      * @return Rebalance timeout.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceTimeout()} instead.
      */
+    @Deprecated
     public long rebalanceTimeout() {
         return ccfg.getRebalanceTimeout();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 58145cf..6a5813a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.cache;
 import java.util.Collection;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture;
@@ -201,4 +203,47 @@ public interface GridCachePreloader {
      * Resume preloader.
      */
     public void resume();
+
+    /**
+     * Rebalance timeout for supply and demand messages in milliseconds.
+     * <p>
+     * The {@link IgniteConfiguration#getRebalanceTimeout()} will be used by default. If an Ignite's configuration value
+     * is not provided than the {@link CacheConfiguration#getRebalanceTimeout()} will be used instead.
+     *
+     * @return Rebalance message timeout in milliseconds.
+     */
+    public long timeout();
+
+    /**
+     * The number of batches generated by supply node at rebalancing procedure start.
+     * <p>
+     * The {@link IgniteConfiguration#getRebalanceBatchesPrefetchCount()} will be used by default. If an Ignite's
+     * configuration value is not provided than the {@link CacheConfiguration#getRebalanceBatchesPrefetchCount()}
+     * will be used instead.
+     *
+     * @return The number of batches prefetch count.
+     */
+    public long batchesPrefetchCount();
+
+    /**
+     * Time in milliseconds to wait between rebalance messages to avoid overloading of CPU or network.
+     * <p>
+     * The {@link IgniteConfiguration#getRebalanceThrottle()} will be used by default. If an Ignite's
+     * configuration value is not provided than the {@link CacheConfiguration#getRebalanceThrottle()}
+     * will be used instead.
+     *
+     * @return Time in milliseconds to wait between rebalance messages, {@code 0} to disable throttling.
+     */
+    public long throttle();
+
+    /**
+     * The supply message size in bytes to be loaded within a single rebalance batch.
+     * <p>
+     * The {@link IgniteConfiguration#getRebalanceBatchSize()} will be used by default. If an Ignite's
+     * configuration value is not provided than the {@link CacheConfiguration#getRebalanceBatchSize()}
+     * will be used instead.
+     *
+     * @return Rebalance message size in bytes.
+     */
+    public int batchSize();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index d176a3b..039067c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -34,6 +34,11 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_THROTTLE;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_TIMEOUT;
+
 /**
  * Adapter for preloading which always assumes that preloading finished.
  */
@@ -190,4 +195,28 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     @Override public void resume() {
         // No-op
     }
+
+    /** {@inheritDoc} */
+    @Override public long timeout() {
+        return grp.shared().gridConfig().getRebalanceTimeout() == DFLT_REBALANCE_TIMEOUT ?
+            grp.config().getRebalanceTimeout() : grp.shared().gridConfig().getRebalanceTimeout();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long batchesPrefetchCount() {
+        return grp.shared().gridConfig().getRebalanceBatchesPrefetchCount() == DFLT_REBALANCE_BATCHES_PREFETCH_COUNT ?
+            grp.config().getRebalanceBatchesPrefetchCount() : grp.shared().gridConfig().getRebalanceBatchesPrefetchCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long throttle() {
+        return grp.shared().gridConfig().getRebalanceThrottle() == DFLT_REBALANCE_THROTTLE ?
+            grp.config().getRebalanceThrottle() : grp.shared().gridConfig().getRebalanceThrottle();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int batchSize() {
+        return grp.shared().gridConfig().getRebalanceBatchSize() == DFLT_REBALANCE_BATCH_SIZE ?
+            grp.config().getRebalanceBatchSize() : grp.shared().gridConfig().getRebalanceBatchSize();
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index cb82a37..167cdcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -527,8 +527,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             throw new IgniteCheckedException("Cannot have more than " + CacheConfiguration.MAX_PARTITIONS_COUNT +
                 " partitions [cacheName=" + cc.getName() + ", partitions=" + cc.getAffinity().partitions() + ']');
 
-        if (cc.getRebalanceMode() != CacheRebalanceMode.NONE)
+        if (cc.getRebalanceMode() != CacheRebalanceMode.NONE) {
             assertParameter(cc.getRebalanceBatchSize() > 0, "rebalanceBatchSize > 0");
+            assertParameter(cc.getRebalanceTimeout() >= 0, "rebalanceTimeout >= 0");
+            assertParameter(cc.getRebalanceThrottle() >= 0, "rebalanceThrottle >= 0");
+            assertParameter(cc.getRebalanceBatchesPrefetchCount() > 0, "rebalanceBatchesPrefetchCount > 0");
+        }
 
         if (cc.getCacheMode() == PARTITIONED || cc.getCacheMode() == REPLICATED) {
             if (cc.getAtomicityMode() == ATOMIC && cc.getWriteSynchronizationMode() == FULL_ASYNC)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index bd7d18f..07d9b64 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -509,7 +509,7 @@ public class GridDhtPartitionDemander {
 
                     demandMsg.topic(rebalanceTopics.get(stripe));
                     demandMsg.rebalanceId(fut.rebalanceId);
-                    demandMsg.timeout(cfg.getRebalanceTimeout());
+                    demandMsg.timeout(grp.preloader().timeout());
 
                     final int topicId = stripe;
 
@@ -837,7 +837,7 @@ public class GridDhtPartitionDemander {
                 supplyMsg.topologyVersion(),
                 grp.groupId());
 
-            d.timeout(grp.config().getRebalanceTimeout());
+            d.timeout(grp.preloader().timeout());
 
             d.topic(rebalanceTopics.get(topicId));
 
@@ -845,7 +845,7 @@ public class GridDhtPartitionDemander {
                 // Send demand message.
                 try {
                     ctx.io().sendOrderedMessage(node, rebalanceTopics.get(topicId),
-                        d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.config().getRebalanceTimeout());
+                        d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.preloader().timeout());
 
                     if (log.isDebugEnabled())
                         log.debug("Send next demand message [" + demandRoutineInfo(topicId, nodeId, supplyMsg) + "]");
@@ -1360,14 +1360,14 @@ public class GridDhtPartitionDemander {
                 this.topologyVersion(),
                 grp.groupId());
 
-            d.timeout(grp.config().getRebalanceTimeout());
+            d.timeout(grp.preloader().timeout());
 
             try {
                 for (int idx = 0; idx < ctx.gridConfig().getRebalanceThreadPoolSize(); idx++) {
                     d.topic(GridCachePartitionExchangeManager.rebalanceTopic(idx));
 
                     ctx.io().sendOrderedMessage(node, GridCachePartitionExchangeManager.rebalanceTopic(idx),
-                        d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.config().getRebalanceTimeout());
+                        d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.preloader().timeout());
                 }
             }
             catch (IgniteCheckedException ignored) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
index e3243ff..b19bddd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
@@ -1,12 +1,12 @@
 /*
  * Copyright 2019 GridGain Systems, Inc. and Contributors.
- * 
+ *
  * Licensed under the GridGain Community Edition License (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at
- * 
+ *
  *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
- * 
+ *
  * 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.
@@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
@@ -74,7 +74,11 @@ class GridDhtPartitionSupplier {
     /** Supply context map. T3: nodeId, topicId, topVer. */
     private final Map<T3<UUID, Integer, AffinityTopologyVersion>, SupplyContext> scMap = new HashMap<>();
 
-    /** Override for rebalance throttle. */
+    /**
+     * Override for rebalance throttle.
+     * @deprecated Use {@link IgniteConfiguration#getRebalanceThrottle()} instead.
+     */
+    @Deprecated
     private long rebalanceThrottleOverride =
         IgniteSystemProperties.getLong(IgniteSystemProperties.IGNITE_REBALANCE_THROTTLE_OVERRIDE, 0);
 
@@ -171,7 +175,7 @@ class GridDhtPartitionSupplier {
      * For each demand message method lookups (or creates new) supply context and starts to iterate entries across requested partitions.
      * Each entry in iterator is placed to prepared supply message.
      *
-     * If supply message size in bytes becomes greater than {@link CacheConfiguration#getRebalanceBatchSize()}
+     * If supply message size in bytes becomes greater than {@link IgniteConfiguration#getRebalanceBatchSize()}
      * method sends this message to demand node and saves partial state of iterated entries to supply context,
      * then restores the context again after new demand message with the same context id is arrived.
      *
@@ -252,7 +256,7 @@ class GridDhtPartitionSupplier {
 
             assert !(sctx != null && !demandMsg.partitions().isEmpty());
 
-            long maxBatchesCnt = grp.config().getRebalanceBatchesPrefetchCount();
+            long maxBatchesCnt = grp.preloader().batchesPrefetchCount();
 
             if (sctx == null) {
                 if (log.isDebugEnabled())
@@ -312,7 +316,7 @@ class GridDhtPartitionSupplier {
                 remainingParts = sctx.remainingParts;
             }
 
-            final int msgMaxSize = grp.config().getRebalanceBatchSize();
+            final int msgMaxSize = grp.preloader().batchSize();
 
             long batchesCnt = 0;
 
@@ -554,8 +558,8 @@ class GridDhtPartitionSupplier {
             // Throttle preloading.
             if (rebalanceThrottleOverride > 0)
                 U.sleep(rebalanceThrottleOverride);
-            else if (grp.config().getRebalanceThrottle() > 0)
-                U.sleep(grp.config().getRebalanceThrottle());
+            else if (grp.preloader().throttle() > 0)
+                U.sleep(grp.preloader().throttle());
 
             return true;
         }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index 882686a..340f342 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -52,7 +52,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
index 55b038a..cb0db54 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
@@ -36,8 +36,8 @@ import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 
 /**
  * Test node restart.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
index 27b9dec..c0fc586 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadBigDataSelfTest.java
@@ -35,8 +35,8 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 
 /**
  * Test large cache counts.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index 20de601..3c09470 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -51,8 +51,8 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.events.EventType.EVTS_CACHE_REBALANCE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
index 6b84b61..b0d11fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
@@ -42,8 +42,8 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
 
 /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
index 2b9c432..adbdc1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
@@ -36,8 +36,8 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_REBALANCE_BATCH_SIZE;
 
 /**
  * Test large cache counts.
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
index d2288d9..fae68ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
@@ -75,7 +75,7 @@ public class ConfigVariations {
         Parameters.parameter("setRebalanceBatchSize", 2028 * 1024),
         Parameters.parameter("setRebalanceBatchesPrefetchCount", 5L),
         Parameters.parameter("setRebalanceThreadPoolSize", 5),
-        Parameters.parameter("setRebalanceTimeout", CacheConfiguration.DFLT_REBALANCE_TIMEOUT * 2),
+        Parameters.parameter("setRebalanceTimeout", IgniteConfiguration.DFLT_REBALANCE_TIMEOUT * 2),
         Parameters.parameter("setRebalanceDelay", 1000L)
     );
 
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
index dddcae6..13f666e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
@@ -71,6 +71,10 @@ namespace Apache.Ignite.Core.Tests.ApiParity
         private static readonly string[] MissingProperties =
         {
             "RebalanceThreadPoolSize",
+            "RebalanceTimeout",
+            "RebalanceBatchesPrefetchCount",
+            "RebalanceThrottle",
+            "RebalanceBatchSize",
             "SegmentationPolicy",
             "isWaitForSegmentOnStart",
             "isAllSegmentationResolversPassRequired",