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

sergeychugunov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 4017911  IGNITE-13345 Warmup feature for caches - IEP-40 part 3 
implementation - Fixes #8148.
4017911 is described below

commit 40179116ab85446740a5b7025cd6cfef05bdefe6
Author: ktkalenko <[email protected]>
AuthorDate: Tue Aug 25 16:45:12 2020 +0300

    IGNITE-13345 Warmup feature for caches - IEP-40 part 3 implementation - 
Fixes #8148.
    
    Signed-off-by: Sergey Chugunov <[email protected]>
---
 .../configuration/DataRegionConfiguration.java     |  26 ++
 .../configuration/DataStorageConfiguration.java    |  27 ++
 .../configuration/LoadAllWarmUpConfiguration.java  |  34 ++
 .../configuration/NoOpWarmUpConfiguration.java     |  33 ++
 .../ignite/configuration/WarmUpConfiguration.java  |  27 ++
 .../org/apache/ignite/internal/IgniteKernal.java   |   9 +-
 .../internal/managers/IgniteMBeansManager.java     |  23 +-
 .../processors/cache/GridCacheProcessor.java       | 113 ++++++-
 .../internal/processors/cache/GridCacheUtils.java  |  42 +++
 .../IgniteCacheDatabaseSharedManager.java          | 105 +++++-
 .../cache/warmup/LoadAllWarmUpStrategy.java        | 259 +++++++++++++++
 .../cache/warmup/NoOpWarmUpStrategy.java           |  51 +++
 .../processors/cache/warmup/WarmUpMXBeanImpl.java  |  58 ++++
 .../processors/cache/warmup/WarmUpStrategy.java    |  51 +++
 .../cache/warmup/WarmUpStrategySupplier.java       |  33 ++
 .../org/apache/ignite/mxbean/WarmUpMXBean.java     |  30 ++
 .../processors/cache/warmup/BlockedWarmUp.java     |  54 +++
 .../cache/warmup/BlockedWarmUpConfiguration.java   |  27 ++
 .../cache/warmup/LoadAllWarmUpConfigurationEx.java |  27 ++
 .../cache/warmup/LoadAllWarmUpStrategyEx.java      |  66 ++++
 .../warmup/LoadAllWarmUpStrategySelfTest.java      | 273 ++++++++++++++++
 .../processors/cache/warmup/Organization.java      |  59 ++++
 .../internal/processors/cache/warmup/Person.java   |  66 ++++
 .../cache/warmup/SimpleObservableWarmUp.java       |  50 +++
 .../SimpleObservableWarmUpConfiguration.java       |  27 ++
 .../processors/cache/warmup/WarmUpSelfTest.java    | 363 +++++++++++++++++++++
 .../cache/warmup/WarmUpTestPluginProvider.java     |  58 ++++
 .../ignite/testsuites/IgnitePdsTestSuite4.java     |   6 +
 28 files changed, 1976 insertions(+), 21 deletions(-)

diff --git 
a/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
index 994b1e0..39c4876 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
@@ -22,6 +22,7 @@ import 
org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
 import org.apache.ignite.mxbean.MetricsMxBean;
+import org.jetbrains.annotations.Nullable;
 
 import static 
org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME;
 
@@ -142,6 +143,9 @@ public final class DataRegionConfiguration implements 
Serializable {
      */
     private boolean lazyMemoryAllocation = true;
 
+    /** Warm-up configuration. */
+    @Nullable private WarmUpConfiguration warmUpCfg;
+
     /**
      * Gets data region name.
      *
@@ -473,6 +477,28 @@ public final class DataRegionConfiguration implements 
Serializable {
         return this;
     }
 
+    /**
+     * Sets warm-up configuration.
+     *
+     * @param warmUpCfg Warm-up configuration. Can be {@code null} (default
+     *      {@link DataStorageConfiguration#getDefaultWarmUpConfiguration} 
will be used).
+     * @return {@code this} for chaining.
+     */
+    public DataRegionConfiguration setWarmUpConfiguration(@Nullable 
WarmUpConfiguration warmUpCfg) {
+        this.warmUpCfg = warmUpCfg;
+
+        return this;
+    }
+
+    /**
+     * Gets warm-up configuration.
+     *
+     * @return Warm-up configuration.
+     */
+    @Nullable public WarmUpConfiguration getWarmUpConfiguration() {
+        return warmUpCfg;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataRegionConfiguration.class, this);
diff --git 
a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
index 8cac348..c1509ef 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.mxbean.MetricsMxBean;
+import org.jetbrains.annotations.Nullable;
 
 import static 
org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE;
 
@@ -306,6 +307,9 @@ public class DataStorageConfiguration implements 
Serializable {
     /** Compression level for WAL page snapshot records. */
     private Integer walPageCompressionLevel;
 
+    /** Default warm-up configuration. */
+    @Nullable private WarmUpConfiguration dfltWarmUpCfg;
+
     /**
      * Creates valid durable memory configuration with all default values.
      */
@@ -1110,6 +1114,29 @@ public class DataStorageConfiguration implements 
Serializable {
         return this;
     }
 
+    /**
+     * Sets default warm-up configuration.
+     *
+     * @param dfltWarmUpCfg Default warm-up configuration. To assign a special
+     *      warm-up configuration for a data region, use
+     *      {@link DataRegionConfiguration#setWarmUpConfiguration}.
+     * @return {@code this} for chaining.
+     */
+    public DataStorageConfiguration setDefaultWarmUpConfiguration(@Nullable 
WarmUpConfiguration dfltWarmUpCfg) {
+        this.dfltWarmUpCfg = dfltWarmUpCfg;
+
+        return this;
+    }
+
+    /**
+     * Gets default warm-up configuration.
+     *
+     * @return Default warm-up configuration.
+     */
+    @Nullable public WarmUpConfiguration getDefaultWarmUpConfiguration() {
+        return dfltWarmUpCfg;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataStorageConfiguration.class, this);
diff --git 
a/modules/core/src/main/java/org/apache/ignite/configuration/LoadAllWarmUpConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/configuration/LoadAllWarmUpConfiguration.java
new file mode 100644
index 0000000..8cb7700
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/configuration/LoadAllWarmUpConfiguration.java
@@ -0,0 +1,34 @@
+/*
+ * 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.configuration;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * "Load All" warm-up configuration that loads data into persistent data region
+ * until it reaches {@link DataRegionConfiguration#getMaxSize} with index 
priority.
+ */
+public class LoadAllWarmUpConfiguration implements WarmUpConfiguration {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(LoadAllWarmUpConfiguration.class, this);
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/configuration/NoOpWarmUpConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/configuration/NoOpWarmUpConfiguration.java
new file mode 100644
index 0000000..455a039
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/configuration/NoOpWarmUpConfiguration.java
@@ -0,0 +1,33 @@
+/*
+ * 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.configuration;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Noop warm-up configuration. Can be used to disable warm-up.
+ */
+public class NoOpWarmUpConfiguration implements WarmUpConfiguration {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(NoOpWarmUpConfiguration.class, this);
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/configuration/WarmUpConfiguration.java
 
b/modules/core/src/main/java/org/apache/ignite/configuration/WarmUpConfiguration.java
new file mode 100644
index 0000000..c02e0bb
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/configuration/WarmUpConfiguration.java
@@ -0,0 +1,27 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ * Warm-up configuration marker interface.
+ */
+public interface WarmUpConfiguration extends Serializable {
+    // No-op.
+}
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 3e0595a..82fcd8a 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
@@ -1264,6 +1264,8 @@ public class IgniteKernal implements IgniteEx, 
IgniteMXBean, Externalizable {
                 if (ctx.config().getPlatformConfiguration() != null)
                     startProcessor(new PlatformPluginProcessor(ctx));
 
+                mBeansMgr.registerMBeansDuringInitPhase();
+
                 ctx.cluster().initDiagnosticListeners();
 
                 fillNodeAttributes(clusterProc.updateNotifierEnabled());
@@ -1355,9 +1357,10 @@ public class IgniteKernal implements IgniteEx, 
IgniteMXBean, Externalizable {
             ctx.cluster().registerMetrics();
 
             // Register MBeans.
-            mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, 
sysExecSvc, stripedExecSvc, p2pExecSvc,
-                mgmtExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, 
idxExecSvc, callbackExecSvc,
-                qryExecSvc, schemaExecSvc, rebalanceExecSvc, 
rebalanceStripedExecSvc, customExecSvcs, ctx.workersRegistry());
+            mBeansMgr.registerMBeansAfterNodeStarted(utilityCachePool, 
execSvc, svcExecSvc, sysExecSvc,
+                stripedExecSvc, p2pExecSvc, mgmtExecSvc, dataStreamExecSvc, 
restExecSvc, affExecSvc, idxExecSvc,
+                callbackExecSvc, qryExecSvc, schemaExecSvc, rebalanceExecSvc, 
rebalanceStripedExecSvc, customExecSvcs,
+                ctx.workersRegistry());
 
             ctx.systemView().registerThreadPools(stripedExecSvc, 
dataStreamExecSvc);
 
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
index 43b5fef..550b60b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.TransactionsMXBeanImpl;
 import org.apache.ignite.internal.managers.encryption.EncryptionMXBeanImpl;
 import 
org.apache.ignite.internal.processors.cache.persistence.DataStorageMXBeanImpl;
 import 
org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMXBeanImpl;
+import org.apache.ignite.internal.processors.cache.warmup.WarmUpMXBeanImpl;
 import 
org.apache.ignite.internal.processors.cluster.BaselineAutoAdjustMXBeanImpl;
 import org.apache.ignite.internal.processors.metric.MetricsMxBeanImpl;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -61,6 +62,7 @@ import org.apache.ignite.mxbean.StripedExecutorMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
 import org.apache.ignite.mxbean.TransactionMetricsMxBean;
 import org.apache.ignite.mxbean.TransactionsMXBean;
+import org.apache.ignite.mxbean.WarmUpMXBean;
 import org.apache.ignite.mxbean.WorkersControlMXBean;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
@@ -91,7 +93,7 @@ public class IgniteMBeansManager {
     }
 
     /**
-     * Registers all kernal MBeans (for kernal, metrics, thread pools).
+     * Registers kernal MBeans (for kernal, metrics, thread pools) after node 
start.
      *
      * @param utilityCachePool Utility cache pool.
      * @param execSvc Executor service.
@@ -113,7 +115,7 @@ public class IgniteMBeansManager {
      * @param workersRegistry Worker registry.
      * @throws IgniteCheckedException if fails to register any of the MBeans.
      */
-    public void registerAllMBeans(
+    public void registerMBeansAfterNodeStarted(
         ExecutorService utilityCachePool,
         final ExecutorService execSvc,
         final ExecutorService svcExecSvc,
@@ -237,6 +239,23 @@ public class IgniteMBeansManager {
     }
 
     /**
+     * Registers kernal MBeans during init phase.
+     *
+     * @throws IgniteCheckedException if fails to register any of the MBeans.
+     */
+    public void registerMBeansDuringInitPhase() throws IgniteCheckedException {
+        if (U.IGNITE_MBEANS_DISABLED)
+            return;
+
+        // Warm-up.
+        registerMBean("WarmUp",
+            WarmUpMXBeanImpl.class.getSimpleName(),
+            new WarmUpMXBeanImpl(ctx.cache()),
+            WarmUpMXBean.class
+        );
+    }
+
+    /**
      * Registers a {@link ThreadPoolMXBean} for an executor.
      *
      * @param name name of the bean to register
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 aec5f58..bd31657 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
@@ -34,6 +34,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
@@ -60,6 +61,7 @@ import 
org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.configuration.WarmUpConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
@@ -127,6 +129,7 @@ import 
org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
 import 
org.apache.ignite.internal.processors.cache.transactions.IgniteTransactionsImpl;
 import 
org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import 
org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.internal.processors.cache.warmup.WarmUpStrategy;
 import 
org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import 
org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
@@ -186,6 +189,9 @@ import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static java.lang.String.format;
+import static java.util.Arrays.asList;
+import static java.util.Objects.isNull;
+import static java.util.Objects.nonNull;
 import static 
org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_REMOVED_ENTRIES_TTL;
 import static 
org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
@@ -981,7 +987,6 @@ public class GridCacheProcessor extends 
GridProcessorAdapter {
      * @param cancel Cancel flag.
      * @param destroy Destroy data flag. Setting to <code>true</code> will 
remove all cache data.
      */
-    @SuppressWarnings({"unchecked"})
     private void stopCache(GridCacheAdapter<?, ?> cache, boolean cancel, 
boolean destroy) {
         stopCache(cache, cancel, destroy, true);
     }
@@ -4472,7 +4477,7 @@ public class GridCacheProcessor extends 
GridProcessorAdapter {
      * @return Cache instance for given name.
      * @throws IgniteCheckedException If failed.
      */
-    @SuppressWarnings({"unchecked", "ConstantConditions"})
+    @SuppressWarnings({"ConstantConditions"})
     public @Nullable <K, V> IgniteCacheProxy<K, V> publicJCache(String 
cacheName,
         boolean failIfNotStarted,
         boolean checkThreadTx) throws IgniteCheckedException {
@@ -5367,6 +5372,12 @@ public class GridCacheProcessor extends 
GridProcessorAdapter {
          */
         private final Map<Integer, QuerySchema> querySchemas = new 
ConcurrentHashMap<>();
 
+        /** Flag for stopping warm-up. */
+        private final AtomicBoolean stopWarmUp = new AtomicBoolean();
+
+        /** Currently running warm-up strategy. */
+        private volatile WarmUpStrategy curWarmUpStrat;
+
         /** {@inheritDoc} */
         @Override public void onBaselineChange() {
             onKernalStopCaches(true);
@@ -5417,7 +5428,13 @@ public class GridCacheProcessor extends 
GridProcessorAdapter {
             IgniteCacheDatabaseSharedManager mgr,
             GridCacheDatabaseSharedManager.RestoreLogicalState restoreState
         ) throws IgniteCheckedException {
-            restorePartitionStates(cacheGroups(), 
restoreState.partitionRecoveryStates());
+            Collection<CacheGroupContext> cacheGrps = cacheGroups();
+
+            restorePartitionStates(cacheGrps, 
restoreState.partitionRecoveryStates());
+
+            // Start warm-up only after restoring memory storage, but before 
starting GridDiscoveryManager.
+            if (!cacheGrps.isEmpty())
+                startWarmUp();
         }
 
         /**
@@ -5483,6 +5500,96 @@ public class GridCacheProcessor extends 
GridProcessorAdapter {
                     ", partitionsProcessed=" + totalProcessed.get() +
                     ", time=" + (U.currentTimeMillis() - startRestorePart) + 
"ms]");
         }
+
+        /**
+         * Start warming up sequentially for each persist data region.
+         *
+         * @throws IgniteCheckedException If failed.
+         */
+        private void startWarmUp() throws IgniteCheckedException {
+            boolean start = false;
+
+            try {
+                // Collecting custom and default data regions.
+                DataStorageConfiguration dsCfg = 
ctx.config().getDataStorageConfiguration();
+
+                List<DataRegionConfiguration> regCfgs =
+                    new 
ArrayList<>(asList(dsCfg.getDefaultDataRegionConfiguration()));
+
+                if (nonNull(dsCfg.getDataRegionConfigurations()))
+                    
regCfgs.addAll(asList(dsCfg.getDataRegionConfigurations()));
+
+                // Warm-up start.
+                Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrats = CU.warmUpStrategies(ctx);
+
+                WarmUpConfiguration dfltWarmUpCfg = 
dsCfg.getDefaultWarmUpConfiguration();
+
+                for (DataRegionConfiguration regCfg : regCfgs) {
+                    if (stopWarmUp.get())
+                        return;
+
+                    if (!regCfg.isPersistenceEnabled())
+                        continue;
+
+                    WarmUpConfiguration warmUpCfg = 
nonNull(regCfg.getWarmUpConfiguration()) ?
+                        regCfg.getWarmUpConfiguration() : dfltWarmUpCfg;
+
+                    if (isNull(warmUpCfg))
+                        continue;
+
+                    WarmUpStrategy warmUpStrat = (curWarmUpStrat = 
warmUpStrats.get(warmUpCfg.getClass()));
+
+                    DataRegion region = 
sharedCtx.database().dataRegion(regCfg.getName());
+
+                    if (!stopWarmUp.get()) {
+                        if (!start && (start = true) && log.isInfoEnabled())
+                            log.info("Warm-up start.");
+
+                        if (log.isInfoEnabled()) {
+                            log.info("Start warm-up for data region [name=" + 
regCfg.getName()
+                                + ", warmUpStrategy=" + warmUpStrat + ", 
warmUpConfig=" + warmUpCfg + ", isDefault="
+                                + (warmUpCfg == dfltWarmUpCfg) + ']');
+                        }
+
+                        warmUpStrat.warmUp(warmUpCfg, region);
+
+                        if (log.isInfoEnabled())
+                            log.info("Finish of warm-up data region: " + 
region.config().getName());
+                    }
+                }
+            }
+            finally {
+                if (stopWarmUp.get() && log.isInfoEnabled())
+                    log.info("Warm-up stop.");
+                else if (start && log.isInfoEnabled())
+                    log.info("Warm-up finish.");
+
+                stopWarmUp.set(true);
+                curWarmUpStrat = null;
+            }
+        }
+    }
+
+    /**
+     * Stop warming up and current running strategy.
+     *
+     * @return {@code true} if stopped by this call.
+     * @throws IgniteCheckedException If there is an error when stopping 
warm-up.
+     */
+    public boolean stopWarmUp() throws IgniteCheckedException {
+        if (recovery.stopWarmUp.compareAndSet(false, true)) {
+            WarmUpStrategy strat = recovery.curWarmUpStrat;
+
+            if (log.isInfoEnabled())
+                log.info("Stopping warm-up strategy: " + strat);
+
+            if (nonNull(strat))
+                strat.stop();
+
+            return true;
+        }
+
+        return false;
     }
 
     /**
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 34b1c9c..0ba96b9 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -59,6 +59,7 @@ import 
org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.configuration.WarmUpConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -78,6 +79,10 @@ import 
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaS
 import 
org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import 
org.apache.ignite.internal.processors.cache.warmup.LoadAllWarmUpStrategy;
+import org.apache.ignite.internal.processors.cache.warmup.NoOpWarmUpStrategy;
+import org.apache.ignite.internal.processors.cache.warmup.WarmUpStrategy;
+import 
org.apache.ignite.internal.processors.cache.warmup.WarmUpStrategySupplier;
 import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.query.QueryUtils;
@@ -112,6 +117,7 @@ import 
org.apache.ignite.transactions.TransactionRollbackException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.Objects.nonNull;
 import static 
org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -2101,6 +2107,42 @@ public class GridCacheUtils {
     }
 
     /**
+     * Getting available warming strategies.
+     *
+     * @param kernalCtx Kernal context.
+     * @return Mapping configuration to strategy.
+     */
+    public static Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrategies(
+        GridKernalContext kernalCtx
+    ) {
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> strategies = 
new HashMap<>();
+
+        // Adding default strategies.
+        WarmUpStrategy[] defStrats = {
+            new NoOpWarmUpStrategy(),
+            new LoadAllWarmUpStrategy(
+                kernalCtx.log(LoadAllWarmUpStrategy.class),
+                () -> kernalCtx.cache().cacheGroups()
+            )
+        };
+
+        for (WarmUpStrategy<?> strategy : defStrats)
+            strategies.putIfAbsent(strategy.configClass(), strategy);
+
+        // Adding strategies from plugins.
+        WarmUpStrategySupplier[] suppliers = 
kernalCtx.plugins().extensions(WarmUpStrategySupplier.class);
+
+        if (nonNull(suppliers)) {
+            for (WarmUpStrategySupplier supplier : suppliers) {
+                for (WarmUpStrategy<?> strategy : supplier.strategies())
+                    strategies.putIfAbsent(strategy.configClass(), strategy);
+            }
+        }
+
+        return strategies;
+    }
+
+    /**
      *
      */
     public interface BackupPostProcessingClosure extends 
IgniteInClosure<Collection<GridCacheEntryInfo>>,
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index 5947039..a147cde 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import javax.management.InstanceNotFoundException;
@@ -41,6 +42,7 @@ import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WarmUpConfiguration;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.GridKernalContext;
@@ -59,7 +61,6 @@ import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import 
org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import 
org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointProgress;
@@ -76,6 +77,7 @@ import 
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaS
 import 
org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener;
 import 
org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import 
org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
+import org.apache.ignite.internal.processors.cache.warmup.WarmUpStrategy;
 import 
org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.util.TimeBag;
 import org.apache.ignite.internal.util.typedef.F;
@@ -84,12 +86,15 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
 import org.apache.ignite.spi.systemview.view.PagesListView;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.Objects.isNull;
+import static java.util.Objects.nonNull;
 import static 
org.apache.ignite.IgniteSystemProperties.IGNITE_REUSE_MEMORY_ON_DEACTIVATE;
 import static 
org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME;
 import static 
org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE;
@@ -527,28 +532,40 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
+     * Validation of memory configuration.
+     *
      * @param memCfg configuration to validate.
+     * @throws IgniteCheckedException In case of validation violation.
      */
     private void validateConfiguration(DataStorageConfiguration memCfg) throws 
IgniteCheckedException {
         checkPageSize(memCfg);
 
         DataRegionConfiguration[] regCfgs = 
memCfg.getDataRegionConfigurations();
 
-        Set<String> regNames = (regCfgs != null) ? 
U.<String>newHashSet(regCfgs.length) : new HashSet<String>(0);
+        Set<String> regNames = new HashSet<>();
 
         checkSystemDataRegionSizeConfiguration(
             memCfg.getSystemRegionInitialSize(),
             memCfg.getSystemRegionMaxSize()
         );
 
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrategies =
+            CU.warmUpStrategies(cctx.kernalContext());
+
         if (regCfgs != null) {
             for (DataRegionConfiguration regCfg : regCfgs)
-                checkDataRegionConfiguration(memCfg, regNames, regCfg);
+                checkDataRegionConfiguration(memCfg, regNames, regCfg, 
warmUpStrategies);
         }
 
-        checkDataRegionConfiguration(memCfg, regNames, 
memCfg.getDefaultDataRegionConfiguration());
+        checkDataRegionConfiguration(memCfg, regNames, 
memCfg.getDefaultDataRegionConfiguration(), warmUpStrategies);
 
         checkWalArchiveSizeConfiguration(memCfg);
+
+        checkExistenceWarmUpConfiguration(
+            memCfg.getDefaultWarmUpConfiguration(),
+            warmUpStrategies,
+            (warmUpCfg) -> "Unknown default warm-up configuration: " + 
warmUpCfg
+        );
     }
 
     /**
@@ -574,12 +591,20 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
+     * Checking configuration of data region.
+     *
      * @param memCfg Mem config.
      * @param regNames Region names.
      * @param regCfg Reg config.
+     * @param warmUpStrategies Available warming-up strategies.
+     * @throws IgniteCheckedException If config is invalid.
      */
-    private void checkDataRegionConfiguration(DataStorageConfiguration memCfg, 
Set<String> regNames,
-        DataRegionConfiguration regCfg) throws IgniteCheckedException {
+    private void checkDataRegionConfiguration(
+        DataStorageConfiguration memCfg,
+        Set<String> regNames,
+        DataRegionConfiguration regCfg,
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrategies
+    ) throws IgniteCheckedException {
         assert regCfg != null;
 
         checkDataRegionName(regCfg.getName(), regNames);
@@ -591,6 +616,8 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
         checkRegionEvictionProperties(regCfg, memCfg);
 
         checkRegionMemoryStorageType(regCfg);
+
+        checkRegionWarmUpConfiguration(regCfg, warmUpStrategies);
     }
 
     /**
@@ -764,10 +791,12 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
-     * @return collection of all configured {@link DataRegion policies}.
+     * Getting registered data regions.
+     *
+     * @return Collection of all configured {@link DataRegion policies}.
      */
     public Collection<DataRegion> dataRegions() {
-        return dataRegionMap != null ? dataRegionMap.values() : null;
+        return dataRegionMap.values();
     }
 
     /**
@@ -810,11 +839,14 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
-     * @param memPlcName data region name.
-     * @return {@link DataRegion} instance associated with a given {@link 
DataRegionConfiguration}.
+     * Getting data region by name.
+     *
+     * @param memPlcName Data region name. In case of {@code null}, default 
data region will be returned.
+     * @return {@link DataRegion} instance associated with a given {@link 
DataRegionConfiguration},
+     *      or {@code null} if there are no registered data regions.
      * @throws IgniteCheckedException in case of request for unknown 
DataRegion.
      */
-    public DataRegion dataRegion(String memPlcName) throws 
IgniteCheckedException {
+    @Nullable public DataRegion dataRegion(@Nullable String memPlcName) throws 
IgniteCheckedException {
         if (memPlcName == null)
             return dfltDataRegion;
 
@@ -897,7 +929,7 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
-     * Clean checkpoint directory {@link 
GridCacheDatabaseSharedManager#cpDir}. The operation
+     * Clean checkpoint directory {@code 
GridCacheDatabaseSharedManager#cpDir}. The operation
      * is necessary when local node joined to baseline topology with different 
consistentId.
      */
     public void cleanupCheckpointDirectory() throws IgniteCheckedException {
@@ -1111,7 +1143,7 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
     }
 
     /**
-     * See {@link GridCacheMapEntry#ensureFreeSpace()}
+     * See {@code GridCacheMapEntry#ensureFreeSpace()}
      *
      * @param memPlc data region.
      */
@@ -1481,4 +1513,51 @@ public class IgniteCacheDatabaseSharedManager extends 
GridCacheSharedManagerAdap
         U.warn(log, "Page-based evictions started." +
                 " Consider increasing 'maxSize' on Data Region configuration: 
" + regCfg.getName());
     }
+
+    /**
+     * Checking existence of a warm-up configuration.
+     *
+     * @param warmUpCfg Warm-up configuration.
+     * @param warmUpStrategies Available warming-up strategies.
+     * @param errMsgSupplier Supplier error message.
+     * @throws IgniteCheckedException If config is invalid.
+     */
+    private void checkExistenceWarmUpConfiguration(
+        @Nullable WarmUpConfiguration warmUpCfg,
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrategies,
+        IgniteClosure<WarmUpConfiguration, String> errMsgSupplier
+    ) throws IgniteCheckedException {
+        if (nonNull(warmUpCfg) && 
!warmUpStrategies.containsKey(warmUpCfg.getClass()))
+            throw new IgniteCheckedException(errMsgSupplier.apply(warmUpCfg));
+    }
+
+    /**
+     * Checking data region warm-up configuration.
+     *
+     * @param regCfg DataRegionConfiguration to validate.
+     * @param warmUpStrategies Available warming-up strategies.
+     * @throws IgniteCheckedException If config is invalid.
+     */
+    private void checkRegionWarmUpConfiguration(
+        DataRegionConfiguration regCfg,
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> 
warmUpStrategies
+    ) throws IgniteCheckedException {
+        WarmUpConfiguration warmUpCfg = regCfg.getWarmUpConfiguration();
+
+        if (isNull(warmUpCfg))
+            return;
+
+        Supplier<String> errPostfix = () -> "[name=" + regCfg.getName() + ", 
warmUpConfig=" + warmUpCfg + ']';
+
+        if (!regCfg.isPersistenceEnabled()) {
+            throw new IgniteCheckedException("Warm-up setting is not expected 
for a non-persistent data region: " +
+                errPostfix.get());
+        }
+
+        checkExistenceWarmUpConfiguration(
+            regCfg.getWarmUpConfiguration(),
+            warmUpStrategies,
+            (warmUpConfig) -> "Unknown data region warm-up configuration: " + 
errPostfix.get()
+        );
+    }
 }
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategy.java
new file mode 100644
index 0000000..71876cc
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategy.java
@@ -0,0 +1,259 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.LoadAllWarmUpConfiguration;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import 
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import 
org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import static java.util.stream.Collectors.toList;
+import static 
org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+
+/**
+ * "Load all" warm-up strategy, which loads pages to persistent data region
+ * until it reaches {@link DataRegionConfiguration#getMaxSize} with priority
+ * to index partitions. Loading occurs sequentially for each cache group,
+ * starting with index partition, and then all others in ascending order.
+ */
+public class LoadAllWarmUpStrategy implements 
WarmUpStrategy<LoadAllWarmUpConfiguration> {
+    /** Logger. */
+    @GridToStringExclude
+    private final IgniteLogger log;
+
+    /**
+     * Cache group contexts supplier.
+     * Since {@link GridCacheProcessor} starts later.
+     */
+    @GridToStringExclude
+    private final Supplier<Collection<CacheGroupContext>> grpCtxSup;
+
+    /** Stop flag. */
+    private volatile boolean stop;
+
+    /**
+     * Constructor.
+     *
+     * @param log Logger.
+     * @param grpCtxSup Cache group contexts supplier. Since {@link 
GridCacheProcessor} starts later.
+     */
+    public LoadAllWarmUpStrategy(IgniteLogger log, 
Supplier<Collection<CacheGroupContext>> grpCtxSup) {
+        this.log = log;
+        this.grpCtxSup = grpCtxSup;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<LoadAllWarmUpConfiguration> configClass() {
+        return LoadAllWarmUpConfiguration.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warmUp(
+        LoadAllWarmUpConfiguration cfg,
+        DataRegion region
+    ) throws IgniteCheckedException {
+        if (stop)
+            return;
+
+        assert region.config().isPersistenceEnabled();
+
+        Map<CacheGroupContext, List<LoadPartition>> loadDataInfo = 
loadDataInfo(region);
+
+        long availableLoadPageCnt = availableLoadPageCount(region);
+
+        if (log.isInfoEnabled()) {
+            Collection<List<LoadPartition>> parts = loadDataInfo.values();
+
+            log.info("Order of cache groups loaded into data region [name=" + 
region.config().getName()
+                + ", partCnt=" + 
parts.stream().mapToLong(Collection::size).sum()
+                + ", pageCnt=" + 
parts.stream().flatMap(Collection::stream).mapToLong(LoadPartition::pages).sum()
+                + ", availablePageCnt=" + availableLoadPageCnt + ", grpNames=" 
+
+                
loadDataInfo.keySet().stream().map(CacheGroupContext::cacheOrGroupName).collect(toList())
 + ']');
+        }
+
+        long loadedPageCnt = 0;
+
+        for (Map.Entry<CacheGroupContext, List<LoadPartition>> e : 
loadDataInfo.entrySet()) {
+            CacheGroupContext grp = e.getKey();
+            List<LoadPartition> parts = e.getValue();
+
+            if (log.isInfoEnabled()) {
+                log.info("Start warm-up cache group, with estimated statistics 
[name=" + grp.cacheOrGroupName()
+                    + ", partCnt=" + parts.size() + ", pageCnt="
+                    + parts.stream().mapToLong(LoadPartition::pages).sum() + 
']');
+            }
+
+            PageMemoryEx pageMemEx = (PageMemoryEx)region.pageMemory();
+
+            for (LoadPartition part : parts) {
+                long pageId = pageMemEx.partitionMetaPageId(grp.groupId(), 
part.part());
+
+                for (int i = 0; i < part.pages(); i++, pageId++, 
loadedPageCnt++) {
+                    if (stop) {
+                        if (log.isInfoEnabled()) {
+                            log.info("Stop warm-up cache group with loaded 
statistics [name="
+                                + grp.cacheOrGroupName() + ", pageCnt=" + 
loadedPageCnt
+                                + ", remainingPageCnt=" + 
(availableLoadPageCnt - loadedPageCnt) + ']');
+                        }
+
+                        return;
+                    }
+
+                    long pagePtr = -1;
+
+                    try {
+                        pagePtr = pageMemEx.acquirePage(grp.groupId(), pageId);
+                    }
+                    finally {
+                        if (pagePtr != -1)
+                            pageMemEx.releasePage(grp.groupId(), pageId, 
pagePtr);
+                    }
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteCheckedException {
+        stop = true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(LoadAllWarmUpStrategy.class, this);
+    }
+
+    /**
+     * Getting count of pages available for loading into data region.
+     *
+     * @param region Data region.
+     * @return Count(non-negative) of pages available for loading into data 
region.
+     */
+    protected long availableLoadPageCount(DataRegion region) {
+        long maxSize = region.config().getMaxSize();
+        long curSize = region.pageMemory().loadedPages() * 
region.pageMemory().systemPageSize();
+
+        return Math.max(0, (maxSize - curSize) / 
region.pageMemory().systemPageSize());
+    }
+
+    /**
+     * Calculation of cache groups, partitions and count of pages that can load
+     * into data region. Calculation starts and includes an index partition for
+     * each group.
+     *
+     * @param region Data region.
+     * @return Loadable groups and partitions.
+     * @throws IgniteCheckedException – if faild.
+     */
+    protected Map<CacheGroupContext, List<LoadPartition>> loadDataInfo(
+        DataRegion region
+    ) throws IgniteCheckedException {
+        // Get cache groups of data region.
+        List<CacheGroupContext> regionGrps = grpCtxSup.get().stream()
+            .filter(grpCtx -> 
region.equals(grpCtx.dataRegion())).collect(toList());
+
+        long availableLoadPageCnt = availableLoadPageCount(region);
+
+        // Computing groups, partitions, and pages to load into data region.
+        Map<CacheGroupContext, List<LoadPartition>> loadableGrps = new 
LinkedHashMap<>();
+
+        for (int i = 0; i < regionGrps.size() && availableLoadPageCnt > 0; 
i++) {
+            CacheGroupContext grp = regionGrps.get(i);
+
+            // Index partition in priority.
+            List<GridDhtLocalPartition> locParts = 
grp.topology().localPartitions();
+
+            for (int j = -1; j < locParts.size() && availableLoadPageCnt > 0; 
j++) {
+                int p = j == -1 ? INDEX_PARTITION : locParts.get(j).id();
+
+                long partPageCnt = 
grp.shared().pageStore().pages(grp.groupId(), p);
+
+                if (partPageCnt > 0) {
+                    long pageCnt = (availableLoadPageCnt - partPageCnt) >= 0 ? 
partPageCnt : availableLoadPageCnt;
+
+                    availableLoadPageCnt -= pageCnt;
+
+                    loadableGrps.computeIfAbsent(grp, grpCtx -> new 
ArrayList<>()).add(new LoadPartition(p, pageCnt));
+                }
+            }
+        }
+
+        return loadableGrps;
+    }
+
+    /**
+     * Information about loaded partition.
+     */
+    static class LoadPartition {
+        /** Partition id. */
+        private final int part;
+
+        /** Number of pages to load. */
+        private final long pages;
+
+        /**
+         * Constructor.
+         *
+         * @param part Partition id.
+         * @param pages Number of pages to load.
+         */
+        public LoadPartition(int part, long pages) {
+            assert part >= 0 : "Partition id cannot be negative.";
+            assert pages > 0 : "Number of pages to load must be greater than 
zero.";
+
+            this.part = part;
+            this.pages = pages;
+        }
+
+        /**
+         * Return partition id.
+         *
+         * @return Partition id.
+         */
+        public int part() {
+            return part;
+        }
+
+        /**
+         * Return number of pages to load.
+         *
+         * @return Number of pages to load.
+         */
+        public long pages() {
+            return pages;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(LoadPartition.class, this);
+        }
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/NoOpWarmUpStrategy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/NoOpWarmUpStrategy.java
new file mode 100644
index 0000000..91fb3ec
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/NoOpWarmUpStrategy.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.NoOpWarmUpConfiguration;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Noop warming up strategy.
+ */
+public class NoOpWarmUpStrategy implements 
WarmUpStrategy<NoOpWarmUpConfiguration> {
+    /** {@inheritDoc} */
+    @Override public Class<NoOpWarmUpConfiguration> configClass() {
+        return NoOpWarmUpConfiguration.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warmUp(
+        NoOpWarmUpConfiguration cfg,
+        DataRegion region
+    ) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(NoOpWarmUpStrategy.class, this);
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpMXBeanImpl.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpMXBeanImpl.java
new file mode 100644
index 0000000..8607baa
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpMXBeanImpl.java
@@ -0,0 +1,58 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.mxbean.WarmUpMXBean;
+
+/**
+ * {@link WarmUpMXBean} implementation.
+ */
+public class WarmUpMXBeanImpl implements WarmUpMXBean {
+    /** Cache processor. */
+    @GridToStringExclude
+    private final GridCacheProcessor cacheProc;
+
+    /**
+     * Constructor.
+     *
+     * @param cacheProc Cache processor.
+     */
+    public WarmUpMXBeanImpl(GridCacheProcessor cacheProc) {
+        this.cacheProc = cacheProc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopWarmUp() {
+        try {
+            cacheProc.stopWarmUp();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(WarmUpMXBeanImpl.class, this);
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategy.java
new file mode 100644
index 0000000..1458193
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategy.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.WarmUpConfiguration;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+
+/**
+ * Interface for warming up.
+ */
+public interface WarmUpStrategy<T extends WarmUpConfiguration> {
+    /**
+     * Returns configuration class for mapping to strategy.
+     *
+     * @return Configuration class.
+     */
+    Class<T> configClass();
+
+    /**
+     * Warm up.
+     *
+     * @param cfg       Warm-up configuration.
+     * @param region    Data region.
+     * @throws IgniteCheckedException if faild.
+     */
+    void warmUp(T cfg, DataRegion region) throws IgniteCheckedException;
+
+    /**
+     * Stop warming up.
+     *
+     * @throws IgniteCheckedException if faild.
+     */
+    void stop() throws IgniteCheckedException;
+}
+
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategySupplier.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategySupplier.java
new file mode 100644
index 0000000..82cef36
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpStrategySupplier.java
@@ -0,0 +1,33 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Collection;
+import org.apache.ignite.plugin.Extension;
+
+/**
+ * Interface for getting warm-up strategies from plugins.
+ */
+public interface WarmUpStrategySupplier extends Extension {
+    /**
+     * Getting warm-up strategies.
+     *
+     * @return Warm-up strategies.
+     */
+    Collection<WarmUpStrategy<?>> strategies();
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/mxbean/WarmUpMXBean.java 
b/modules/core/src/main/java/org/apache/ignite/mxbean/WarmUpMXBean.java
new file mode 100644
index 0000000..ec82181
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/WarmUpMXBean.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mxbean;
+
+/**
+ * Warm-up MXBean interface.
+ */
+@MXBeanDescription("MBean that provides access to warm-up.")
+public interface WarmUpMXBean {
+    /**
+     * Stop warm-up.
+     */
+    @MXBeanDescription("Stop warm-up.")
+    void stopWarmUp();
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUp.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUp.java
new file mode 100644
index 0000000..25a1a3b
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUp.java
@@ -0,0 +1,54 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Warm-up strategy that only waits for {@link #stop} call.
+ */
+class BlockedWarmUp implements WarmUpStrategy<BlockedWarmUpConfiguration> {
+    /** Stop latch. */
+    final CountDownLatch stopLatch = new CountDownLatch(1);
+
+    /** Start latch. */
+    final CountDownLatch startLatch = new CountDownLatch(1);
+
+    /** {@inheritDoc} */
+    @Override public Class<BlockedWarmUpConfiguration> configClass() {
+        return BlockedWarmUpConfiguration.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warmUp(
+        BlockedWarmUpConfiguration cfg,
+        DataRegion region
+    ) throws IgniteCheckedException {
+        startLatch.countDown();
+
+        U.await(stopLatch);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteCheckedException {
+        stopLatch.countDown();
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUpConfiguration.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUpConfiguration.java
new file mode 100644
index 0000000..079021c
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/BlockedWarmUpConfiguration.java
@@ -0,0 +1,27 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.configuration.WarmUpConfiguration;
+
+/**
+ * Configuration for {@link BlockedWarmUp}.
+ */
+class BlockedWarmUpConfiguration implements WarmUpConfiguration {
+    // No-op.
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpConfigurationEx.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpConfigurationEx.java
new file mode 100644
index 0000000..b22f566
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpConfigurationEx.java
@@ -0,0 +1,27 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.configuration.LoadAllWarmUpConfiguration;
+
+/**
+ * Extension {@link LoadAllWarmUpConfiguration}.
+ */
+class LoadAllWarmUpConfigurationEx extends LoadAllWarmUpConfiguration {
+    // No-op.
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategyEx.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategyEx.java
new file mode 100644
index 0000000..ed24e35
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategyEx.java
@@ -0,0 +1,66 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+
+import static java.util.Objects.nonNull;
+
+/**
+ * Extension {@link LoadAllWarmUpStrategy}.
+ */
+class LoadAllWarmUpStrategyEx extends LoadAllWarmUpStrategy {
+    /** {@link #loadDataInfo} callback. */
+    static volatile BiConsumer<String, Map<CacheGroupContext, 
List<LoadPartition>>> loadDataInfoCb;
+
+    /**
+     * Constructor.
+     *
+     * @param log       Logger.
+     * @param grpCtxSup Cache group contexts supplier. Since {@link 
GridCacheProcessor} starts later.
+     */
+    public LoadAllWarmUpStrategyEx(IgniteLogger log, 
Supplier<Collection<CacheGroupContext>> grpCtxSup) {
+        super(log, grpCtxSup);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class configClass() {
+        return LoadAllWarmUpConfigurationEx.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Map<CacheGroupContext, List<LoadPartition>> 
loadDataInfo(
+        DataRegion region
+    ) throws IgniteCheckedException {
+        Map<CacheGroupContext, List<LoadPartition>> loadDataInfo = 
super.loadDataInfo(region);
+
+        if (nonNull(loadDataInfoCb))
+            loadDataInfoCb.accept(region.config().getName(), loadDataInfo);
+
+        return loadDataInfo;
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategySelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategySelfTest.java
new file mode 100644
index 0000000..9a318b5
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/LoadAllWarmUpStrategySelfTest.java
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import 
org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import 
org.apache.ignite.internal.processors.cache.warmup.LoadAllWarmUpStrategy.LoadPartition;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+import static java.util.Collections.emptyList;
+import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.toMap;
+
+/**
+ * Test class for testing {@link LoadAllWarmUpStrategy}.
+ */
+public class LoadAllWarmUpStrategySelfTest extends GridCommonAbstractTest {
+    /** Flag for enabling warm-up. */
+    private boolean warmUp;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        LoadAllWarmUpStrategyEx.loadDataInfoCb = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String 
igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setPluginProviders(new WarmUpTestPluginProvider())
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new 
DataRegionConfiguration().setName("dr_0").setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(!warmUp ? null : new 
LoadAllWarmUpConfigurationEx()),
+                        new 
DataRegionConfiguration().setName("dr_1").setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(!warmUp ? null : new 
LoadAllWarmUpConfigurationEx())
+                    )
+            ).setCacheConfiguration(
+                cacheCfg("c_0", "g_0", "dr_0", Organization.queryEntity()),
+                cacheCfg("c_1", "g_0", "dr_0", Person.queryEntity()),
+                cacheCfg("c_2", "g_1", "dr_1", Organization.queryEntity()),
+                cacheCfg("c_3", "g_1", "dr_1", Person.queryEntity())
+            );
+    }
+
+    /**
+     * Test checks that number of pages loaded is equal to number of pages 
warmed up.
+     * <p/>
+     * Steps:
+     * 1)Start a node with static and dynamic caches and fill them in;
+     * 2)Make a checkpoint and get number of pages loaded;
+     * 3)Restart node and get number of pages warmed up;
+     * 4)Check that number of loaded and warmed pages is equal;
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testSimple() throws Exception {
+        IgniteEx n = startGrid(0);
+        n.cluster().state(ClusterState.ACTIVE);
+
+        IgniteCache c4 = n.getOrCreateCache(cacheCfg("c_4", "g_2", "dr_0"));
+
+        for (int i = 0; i < 5_000; i++) {
+            n.cache("c_0").put("c_0" + i, new Organization(i, "c_0" + i));
+            n.cache("c_1").put("c_1" + i, new Person(i, "c_1" + i, i));
+            n.cache("c_2").put("c_2" + i, new Organization(i, "c_2" + i));
+            n.cache("c_3").put("c_3" + i, new Person(i, "c_3" + i, i));
+
+            c4.put("c_4" + i, ThreadLocalRandom.current().nextInt());
+        }
+
+        forceCheckpoint();
+
+        Map<String, Long> expLoadedPages = loadedeDataRegionPages(n);
+
+        stopAllGrids();
+
+        warmUp = true;
+
+        n = startGrid(0);
+
+        Map<String, Long> actLoadedPages = loadedeDataRegionPages(n);
+
+        assertEquals(expLoadedPages.size(), actLoadedPages.size());
+
+        expLoadedPages.forEach((regName, loadedPages) -> {
+            assertTrue(regName, actLoadedPages.containsKey(regName));
+            assertEquals(regName, loadedPages, actLoadedPages.get(regName));
+        });
+    }
+
+    /**
+     * Test checks that if memory is less than pds, not all pages in pds will 
warm-up.
+     * There may be evictions during warm-up, so count of pages loaded is not 
maximum.
+     * <p/>
+     * Steps:
+     * 1)Start node and fill it with data for first data region until it is 2 
* {@code MIN_PAGE_MEMORY_SIZE};
+     * 2)Make a checkpoint;
+     * 3)Restart node with warm-up, change maximum data region size to {@code 
MIN_PAGE_MEMORY_SIZE},
+     * and listen for {@link LoadAllWarmUpStrategyEx#loadDataInfo};
+     * 4)Check that estimated count of pages to warm-up is between maximum and
+     * approximate minimum count of pages to load;
+     * 5)Checking that total count of pages loaded is between maximum and
+     * approximate minimum count of pages to load.
+     *
+     * Approximate value due to fact that there are already loaded pages at
+     * beginning of warm-up, as well as evictions occur during warm-up.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMemoryLessPds() throws Exception {
+        IgniteEx n = startGrid(0);
+        n.cluster().state(ClusterState.ACTIVE);
+
+        int i = 0;
+
+        final long minMemSize = 
U.field(IgniteCacheDatabaseSharedManager.class, "MIN_PAGE_MEMORY_SIZE");
+
+        DataRegion dr_0 = 
n.context().cache().context().database().dataRegion("dr_0");
+
+        while (dr_0.pageMemory().loadedPages() * 
dr_0.pageMemory().systemPageSize() < 2 * minMemSize) {
+            n.cache("c_0").put("c_0" + i, new Organization(i, "c_0" + i));
+            n.cache("c_1").put("c_1" + i, new Person(i, "c_1" + i, i));
+
+            i++;
+        }
+
+        forceCheckpoint();
+
+        stopAllGrids();
+
+        warmUp = true;
+
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0));
+        
cfg.getDataStorageConfiguration().getDataRegionConfigurations()[0].setMaxSize(minMemSize);
+
+        Map<String, Map<CacheGroupContext, List<LoadPartition>>> 
loadDataInfoMap = new ConcurrentHashMap<>();
+
+        LoadAllWarmUpStrategyEx.loadDataInfoCb = loadDataInfoMap::put;
+
+        n = startGrid(cfg);
+
+        dr_0 = n.context().cache().context().database().dataRegion("dr_0");
+
+        long warmUpPageCnt = 
loadDataInfoMap.get("dr_0").values().stream().flatMap(Collection::stream).mapToLong(LoadPartition::pages).sum();
+        long maxLoadPages = minMemSize / dr_0.pageMemory().systemPageSize();
+        long minLoadPages = maxLoadPages - 100;
+        long loadPages = dr_0.pageMemory().loadedPages();
+
+        // There are loaded pages before warm-up.
+        assertTrue(warmUpPageCnt >= minLoadPages && warmUpPageCnt <= 
maxLoadPages);
+
+        // Pages may be evicted.
+        assertTrue(loadPages >= minLoadPages && loadPages <= maxLoadPages);
+    }
+
+    /**
+     * Create cache configuration.
+     *
+     * @param name Cache name.
+     * @param grpName Cache group name.
+     * @param regName Data region name.
+     * @param qryEntities Query entities.
+     * @return New cache configuration.
+     */
+    private CacheConfiguration cacheCfg(String name, String grpName, String 
regName, QueryEntity... qryEntities) {
+        requireNonNull(name);
+        requireNonNull(grpName);
+        requireNonNull(regName);
+
+        return new CacheConfiguration(name)
+            .setGroupName(grpName)
+            .setDataRegionName(regName)
+            .setAffinity(new GapRendezvousAffinityFunction(false, 5))
+            .setQueryEntities(Arrays.asList(qryEntities));
+    }
+
+    /**
+     * Counting of loaded pages for data regions.
+     *
+     * @param n Node.
+     * @return Mapping: {dataRegionName -> loadedPageCnt}.
+     */
+    private Map<String, Long> loadedeDataRegionPages(IgniteEx n) {
+        requireNonNull(n);
+
+        return n.context().cache().cacheGroups().stream()
+            .filter(grpCtx -> grpCtx.userCache() && 
grpCtx.persistenceEnabled())
+            // Check for exists gap in local partitions.
+            .peek(grpCtx -> 
assertTrue(grpCtx.topology().localPartitions().size() < 
grpCtx.topology().partitions()))
+            .map(CacheGroupContext::dataRegion)
+            .distinct()
+            .collect(toMap(region -> region.config().getName(), region -> 
region.pageMemory().loadedPages()));
+    }
+
+    /**
+     * {@link RendezvousAffinityFunction} for presence of a gap partition.
+     */
+    private static class GapRendezvousAffinityFunction extends 
RendezvousAffinityFunction {
+        /** Gap partition id. */
+        public static final int GAP_PART = 2;
+
+        /**
+         * Constructor that invoke {@link 
RendezvousAffinityFunction#RendezvousAffinityFunction(boolean, int)}.
+         */
+        public GapRendezvousAffinityFunction(boolean exclNeighbors, int parts) 
{
+            super(exclNeighbors, parts);
+
+            assert parts > GAP_PART : parts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<ClusterNode> assignPartition(
+            int part,
+            List<ClusterNode> nodes,
+            int backups,
+            @Nullable Map<UUID, Collection<ClusterNode>> neighborhoodCache
+        ) {
+            return part == GAP_PART ? emptyList() : 
super.assignPartition(part, nodes, backups, neighborhoodCache);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int partition(Object key) {
+            int part = super.partition(key);
+
+            return part == GAP_PART ? GAP_PART - 1 : part;
+        }
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Organization.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Organization.java
new file mode 100644
index 0000000..13e10ef
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Organization.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.processors.cache.warmup;
+
+import java.util.Arrays;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+
+/**
+ * Organization.
+ */
+class Organization {
+    /** Id. */
+    final long id;
+
+    /** Name. */
+    final String name;
+
+    /**
+     * Constructor.
+     *
+     * @param id Id.
+     * @param name Name.
+     */
+    Organization(long id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    /**
+     * Create query entity.
+     *
+     * @return New query entity.
+     */
+    static QueryEntity queryEntity() {
+        return new QueryEntity(String.class, Organization.class)
+            .addQueryField("id", Long.class.getName(), null)
+            .addQueryField("name", String.class.getName(), null)
+            .setIndexes(Arrays.asList(
+                new QueryIndex("id"),
+                new QueryIndex("name")
+            ));
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Person.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Person.java
new file mode 100644
index 0000000..7bbf6ec
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/Person.java
@@ -0,0 +1,66 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Arrays;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+
+/**
+ * Person.
+ */
+class Person {
+    /** Id. */
+    final long id;
+
+    /** Name. */
+    final String name;
+
+    /** Organization id. */
+    final long orgId;
+
+    /**
+     * Constructor.
+     *
+     * @param id Id.
+     * @param name Name.
+     * @param orgId Organization id.
+     */
+    Person(long id, String name, long orgId) {
+        this.id = id;
+        this.name = name;
+        this.orgId = orgId;
+    }
+
+    /**
+     * Create query entity.
+     *
+     * @return New query entity.
+     */
+    static QueryEntity queryEntity() {
+        return new QueryEntity(String.class, Person.class)
+            .addQueryField("id", Long.class.getName(), null)
+            .addQueryField("name", String.class.getName(), null)
+            .addQueryField("orgId", Long.class.getName(), null)
+            .setIndexes(Arrays.asList(
+                new QueryIndex("id"),
+                new QueryIndex("name"),
+                new QueryIndex("orgId")
+            ));
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUp.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUp.java
new file mode 100644
index 0000000..e743dd5
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUp.java
@@ -0,0 +1,50 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+
+/**
+ * Warm-up strategy that only records which regions have visited it and how 
many times.
+ */
+class SimpleObservableWarmUp implements 
WarmUpStrategy<SimpleObservableWarmUpConfiguration> {
+    /** Visited regions with a counter. */
+    final Map<String, AtomicInteger> visitRegions = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public Class<SimpleObservableWarmUpConfiguration> configClass() {
+        return SimpleObservableWarmUpConfiguration.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warmUp(
+        SimpleObservableWarmUpConfiguration cfg,
+        DataRegion region
+    ) throws IgniteCheckedException {
+        visitRegions.computeIfAbsent(region.config().getName(), s -> new 
AtomicInteger()).incrementAndGet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteCheckedException {
+        // No-op.
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUpConfiguration.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUpConfiguration.java
new file mode 100644
index 0000000..d050133
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/SimpleObservableWarmUpConfiguration.java
@@ -0,0 +1,27 @@
+/*
+ * 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.processors.cache.warmup;
+
+import org.apache.ignite.configuration.WarmUpConfiguration;
+
+/**
+ * Configuration for {@link SimpleObservableWarmUp}.
+ */
+class SimpleObservableWarmUpConfiguration implements WarmUpConfiguration {
+    // No-op.
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpSelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpSelfTest.java
new file mode 100644
index 0000000..55acf79
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpSelfTest.java
@@ -0,0 +1,363 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NoOpWarmUpConfiguration;
+import org.apache.ignite.configuration.WarmUpConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.mxbean.WarmUpMXBean;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static java.util.Objects.requireNonNull;
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+
+/**
+ * Class for testing warm-up.
+ */
+public class WarmUpSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String 
igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new DataRegionConfiguration()
+                            .setName("persist")
+                            .setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(new 
NoOpWarmUpConfiguration())
+                    )
+            ).setPluginProviders(new WarmUpTestPluginProvider());
+    }
+
+    /**
+     * Test checks that an unknown default warm-up configuration cannot be 
passed.
+     * <p>
+     * Steps:
+     * 1)Adding an unknown warm-up configuration to {@link 
DataStorageConfiguration};
+     * 2)Starting node and getting an error.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testUnknownDefaultWarmUpConfiguration() throws Exception {
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultWarmUpConfiguration(new WarmUpConfiguration() {
+                    })
+            );
+
+        assertThrowsAnyCause(
+            log,
+            () -> startGrid(cfg),
+            IgniteCheckedException.class,
+            "Unknown default warm-up configuration"
+        );
+    }
+
+    /**
+     * Test checks that an unknown data region warm-up configuration cannot be 
passed.
+     * <p>
+     * Steps:
+     * 1)Adding an unknown warm-up configuration to {@link 
DataRegionConfiguration};
+     * 2)Starting node and getting an error.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testUnknownDataRegionWarmUpConfiguration() throws Exception {
+        String regName = "error";
+
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new DataRegionConfiguration()
+                            .setName(regName)
+                            .setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(new WarmUpConfiguration() {
+                            })
+                    )
+            );
+
+        assertThrowsAnyCause(
+            log,
+            () -> startGrid(cfg),
+            IgniteCheckedException.class,
+            "Unknown data region warm-up configuration: [name=" + regName
+        );
+    }
+
+    /**
+     * Test checks that an unknown data region warm-up configuration cannot be 
passed.
+     * <p>
+     * Steps:
+     * 1)Adding an warm-up configuration to non-persistent {@link 
DataRegionConfiguration};
+     * 2)Starting node and getting an error.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNonPersistentDataRegionWarmUpConfiguration() throws 
Exception {
+        String regName = "error";
+
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new DataRegionConfiguration()
+                            .setName(regName)
+                            .setWarmUpConfiguration(new 
NoOpWarmUpConfiguration())
+                    )
+            );
+
+        assertThrowsAnyCause(
+            log,
+            () -> startGrid(cfg),
+            IgniteCheckedException.class,
+            "Warm-up setting is not expected for a non-persistent data region: 
[name=" + regName
+        );
+    }
+
+    /**
+     * Test verifies that available warm-up strategies are correct.
+     * <p>
+     * Steps:
+     * 1)Starting a node, without plugins;
+     * 2)Check that only basic strategies are available;
+     * 3)Restarting a node with a test plugin containing additional strategies;
+     * 4)Checking that basic + from plugin strategies are available.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAvailableWarmUpStrategies() throws Exception {
+        IgniteEx n = 
startGrid(getConfiguration(getTestIgniteInstanceName(0)).setPluginProviders());
+
+        GridCacheProcessor cacheProc = n.context().cache();
+
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> expStrats =
+            Stream.of(new NoOpWarmUpStrategy(), new LoadAllWarmUpStrategy(log, 
cacheProc::cacheGroups))
+                .collect(toMap(WarmUpStrategy::configClass, identity()));
+
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> actStrats = 
CU.warmUpStrategies(n.context());
+
+        assertEquals(expStrats, actStrats);
+
+        stopAllGrids();
+
+        n = startGrid(0);
+
+        WarmUpTestPluginProvider pluginProvider = 
(WarmUpTestPluginProvider)n.configuration().getPluginProviders()[0];
+
+        pluginProvider.strats.forEach(strat -> 
assertNull(expStrats.put(strat.configClass(), strat)));
+
+        actStrats = CU.warmUpStrategies(n.context());
+
+        assertEquals(expStrats, actStrats);
+    }
+    
+    /**
+     * Test checks that strategies are executed according to configuration.
+     * <p>
+     * Steps:
+     * 1)Starting a node with a single region that has been configured for 
{@link SimpleObservableWarmUp};
+     * 2)Check that strategy was executed only for it region;
+     * 3)Restarting node with default {@link 
SimpleObservableWarmUpConfiguration};
+     * 4)Checks that {@link SimpleObservableWarmUp} was only executed for 
persistent regions
+     * that were not configured by {@link SimpleObservableWarmUpConfiguration}.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testExecutionStrategies() throws Exception {
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new DataRegionConfiguration().setName("0"),
+                        new 
DataRegionConfiguration().setName("1").setPersistenceEnabled(true),
+                        new 
DataRegionConfiguration().setName("2").setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(new 
SimpleObservableWarmUpConfiguration())
+                    )
+            );
+
+        startGrid(cfg);
+
+        WarmUpTestPluginProvider pluginProvider = 
(WarmUpTestPluginProvider)cfg.getPluginProviders()[0];
+        SimpleObservableWarmUp observableWarmUp = 
(SimpleObservableWarmUp)pluginProvider.strats.get(0);
+
+        assertEquals(1, observableWarmUp.visitRegions.size());
+        assertTrue(observableWarmUp.visitRegions.containsKey("2"));
+        assertEquals(1, observableWarmUp.visitRegions.get("2").get());
+
+        stopAllGrids();
+
+        cfg = getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultWarmUpConfiguration(new 
SimpleObservableWarmUpConfiguration())
+                    .setDataRegionConfigurations(
+                        new DataRegionConfiguration().setName("0"),
+                        new 
DataRegionConfiguration().setName("1").setPersistenceEnabled(true),
+                        new 
DataRegionConfiguration().setName("2").setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(new 
NoOpWarmUpConfiguration())
+                    ).setDefaultDataRegionConfiguration(
+                    new 
DataRegionConfiguration().setName("3").setPersistenceEnabled(true)
+                )
+            );
+
+        startGrid(cfg);
+
+        pluginProvider = (WarmUpTestPluginProvider)cfg.getPluginProviders()[0];
+        observableWarmUp = 
(SimpleObservableWarmUp)pluginProvider.strats.get(0);
+
+        assertEquals(2, observableWarmUp.visitRegions.size());
+
+        assertTrue(observableWarmUp.visitRegions.containsKey("1"));
+        assertTrue(observableWarmUp.visitRegions.containsKey("3"));
+
+        assertEquals(1, observableWarmUp.visitRegions.get("1").get());
+        assertEquals(1, observableWarmUp.visitRegions.get("3").get());
+    }
+
+    /**
+     * Test checks to stop warming up.
+     * <p>
+     * Steps:
+     * 1)Running a node in a separate thread with {@link 
BlockedWarmUpConfiguration} for one region;
+     * 2)Stop warm-up;
+     * 3)Make sure that warm-up is stopped and node has started successfully.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testStopWarmUp() throws Exception {
+        checkStopWarmUp(new IgniteInClosureX<IgniteKernal>() {
+            /** {@inheritDoc} */
+            @Override public void applyx(IgniteKernal kernal) throws 
IgniteCheckedException {
+                assertTrue(kernal.context().cache().stopWarmUp());
+                assertFalse(kernal.context().cache().stopWarmUp());
+            }
+        });
+    }
+
+    /**
+     * Test checks to stop warming up by MXBean.
+     * <p>
+     * Steps:
+     * 1)Running a node in a separate thread with {@link 
BlockedWarmUpConfiguration} for one region;
+     * 2)Stop warm-up by MXBean;
+     * 3)Make sure that warm-up is stopped and node has started successfully.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testStopWarmUpByMXBean() throws Exception {
+        checkStopWarmUp(new IgniteInClosureX<IgniteKernal>() {
+            /** {@inheritDoc} */
+            @Override public void applyx(IgniteKernal kernal) throws 
IgniteCheckedException {
+                WarmUpMXBean warmUpMXBean = getMxBean(
+                    kernal.configuration().getIgniteInstanceName(),
+                    "WarmUp",
+                    WarmUpMXBeanImpl.class.getSimpleName(),
+                    WarmUpMXBean.class
+                );
+
+                warmUpMXBean.stopWarmUp();
+            }
+        });
+    }
+
+    /**
+     * Asserts that two warm-up maps are equal.
+     *
+     * @param exp Expected value.
+     * @param act Actual value
+     */
+    private void assertEquals(
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> exp,
+        Map<Class<? extends WarmUpConfiguration>, WarmUpStrategy> act
+    ) {
+        assertEquals(exp.size(), act.size());
+
+        exp.forEach((cfgCls, strat) -> {
+            assertTrue(cfgCls.toString(), act.containsKey(cfgCls));
+            assertTrue(cfgCls.toString(), act.get(cfgCls).getClass() == 
strat.getClass());
+        });
+    }
+
+    /**
+     * Checking correct warm-up stop.
+     *
+     * @param stopCX Stop warm-up function.
+     * @throws Exception If failed.
+     */
+    private void checkStopWarmUp(IgniteInClosureX<IgniteKernal> stopCX) throws 
Exception {
+        requireNonNull(stopCX);
+
+        IgniteConfiguration cfg = 
getConfiguration(getTestIgniteInstanceName(0))
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDataRegionConfigurations(
+                        new 
DataRegionConfiguration().setName("1").setPersistenceEnabled(true)
+                            .setWarmUpConfiguration(new 
BlockedWarmUpConfiguration())
+                    )
+            );
+
+        IgniteInternalFuture<IgniteEx> stratFut = GridTestUtils.runAsync(() -> 
startGrid(cfg));
+
+        WarmUpTestPluginProvider pluginProvider = 
(WarmUpTestPluginProvider)cfg.getPluginProviders()[0];
+        BlockedWarmUp strat = (BlockedWarmUp)pluginProvider.strats.get(1);
+
+        strat.startLatch.await(1, TimeUnit.MINUTES);
+
+        IgniteKernal n = IgnitionEx.gridx(cfg.getIgniteInstanceName());
+
+        stopCX.apply(n);
+
+        assertEquals(0, strat.stopLatch.getCount());
+        assertNotNull(stratFut.get(TimeUnit.MINUTES.toMillis(1)));
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpTestPluginProvider.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpTestPluginProvider.java
new file mode 100644
index 0000000..ca8ff86
--- /dev/null
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/warmup/WarmUpTestPluginProvider.java
@@ -0,0 +1,58 @@
+/*
+ * 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.processors.cache.warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.plugin.AbstractTestPluginProvider;
+import org.apache.ignite.plugin.ExtensionRegistry;
+import org.apache.ignite.plugin.PluginContext;
+
+/**
+ * Test plugin provider for test strategies.
+ */
+class WarmUpTestPluginProvider extends AbstractTestPluginProvider {
+    /** Collection of strategies. */
+    final List<WarmUpStrategy<?>> strats = new ArrayList<>(Arrays.asList(
+        new SimpleObservableWarmUp(),
+        new BlockedWarmUp()
+    ));
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return getClass().getSimpleName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initExtensions(PluginContext ctx, ExtensionRegistry 
registry) {
+        super.initExtensions(ctx, registry);
+
+        IgniteEx gridx = (IgniteEx)ctx.grid();
+
+        strats.add(new LoadAllWarmUpStrategyEx(gridx.log(), () -> 
gridx.context().cache().cacheGroups()));
+
+        registry.registerExtension(WarmUpStrategySupplier.class, new 
WarmUpStrategySupplier() {
+            /** {@inheritDoc} */
+            @Override public Collection<WarmUpStrategy<?>> strategies() {
+                return strats;
+            }
+        });
+    }
+}
diff --git 
a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
 
b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
index 18b483a..ee76c08 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
@@ -54,6 +54,8 @@ import 
org.apache.ignite.internal.processors.cache.persistence.diagnostic.pagelo
 import 
org.apache.ignite.internal.processors.cache.persistence.diagnostic.pagelocktracker.stack.HeapArrayLockStackTest;
 import 
org.apache.ignite.internal.processors.cache.persistence.diagnostic.pagelocktracker.stack.OffHeapLockStackTest;
 import 
org.apache.ignite.internal.processors.cache.persistence.file.FileDownloaderTest;
+import 
org.apache.ignite.internal.processors.cache.warmup.LoadAllWarmUpStrategySelfTest;
+import org.apache.ignite.internal.processors.cache.warmup.WarmUpSelfTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.DynamicSuite;
 import org.junit.runner.RunWith;
@@ -113,6 +115,10 @@ public class IgnitePdsTestSuite4 {
 
         GridTestUtils.addTestIfNeeded(suite, 
IgnitePdsConsistencyOnDelayedPartitionOwning.class, ignoredTests);
 
+        // Warm-up tests.
+        GridTestUtils.addTestIfNeeded(suite, WarmUpSelfTest.class, 
ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, 
LoadAllWarmUpStrategySelfTest.class, ignoredTests);
+
         return suite;
     }
 

Reply via email to