IGNITE-13581 Change Data Capture implementation (#9054)

diff --git a/bin/ignite-cdc.sh b/bin/ignite-cdc.sh
new file mode 100755
index 0000000..b04345d
--- /dev/null
+++ b/bin/ignite-cdc.sh
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+#
+
+export MAIN_CLASS="org.apache.ignite.startup.cmdline.CdcCommandLineStartup"
+
+if [ "${IGNITE_HOME:-}" = "" ];
+    then IGNITE_HOME_TMP="$(dirname "$(cd "$(dirname "$0")"; "pwd")")";
+    else IGNITE_HOME_TMP=${IGNITE_HOME};
+fi
+
+${IGNITE_HOME_TMP}/bin/ignite.sh "$@"
diff --git a/config/ignite-log4j2.xml b/config/ignite-log4j2.xml
index b55c563..5beee0b 100644
--- a/config/ignite-log4j2.xml
+++ b/config/ignite-log4j2.xml
@@ -31,8 +31,8 @@
         <Routing name="FILE">
             <Routes pattern="$${sys:nodeId}">
                 <Route>
-                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}.log"
-                                 filePattern="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
+                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}.log"
+                                 filePattern="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
                         <PatternLayout pattern="[%d{ISO8601}][%-5p][%t][%c{1}]%notEmpty{[%markerSimpleName]} %m%n"/>
                         <Policies>
                             <TimeBasedTriggeringPolicy interval="6" modulate="true" />
diff --git a/config/java.util.logging.properties b/config/java.util.logging.properties
index f68af5b..3e34a50 100644
--- a/config/java.util.logging.properties
+++ b/config/java.util.logging.properties
@@ -68,7 +68,7 @@
 # under `$IGNITE_HOME/work/log/` directory. The placeholder `%{id8}` is a truncated node ID.
 #
 org.apache.ignite.logger.java.JavaLoggerFileHandler.formatter=org.apache.ignite.logger.java.JavaLoggerFormatter
-org.apache.ignite.logger.java.JavaLoggerFileHandler.pattern=ignite-%{id8}.%g.log
+org.apache.ignite.logger.java.JavaLoggerFileHandler.pattern=%{app}-%{id8}.%g.log
 org.apache.ignite.logger.java.JavaLoggerFileHandler.level=INFO
 org.apache.ignite.logger.java.JavaLoggerFileHandler.limit=10485760
 org.apache.ignite.logger.java.JavaLoggerFileHandler.count=10
diff --git a/docs/_docs/code-snippets/xml/log4j2-config.xml b/docs/_docs/code-snippets/xml/log4j2-config.xml
index 2b41228..1728ed9 100644
--- a/docs/_docs/code-snippets/xml/log4j2-config.xml
+++ b/docs/_docs/code-snippets/xml/log4j2-config.xml
@@ -29,8 +29,8 @@
         <Routing name="FILE">
             <Routes pattern="$${sys:nodeId}">
                 <Route>
-                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}.log"
-                                 filePattern="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
+                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}.log"
+                                 filePattern="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
                         <PatternLayout pattern="[%d{ISO8601}][%-5p][%t][%c{1}]%notEmpty{[%markerSimpleName]} %m%n"/>
                         <Policies>
                             <TimeBasedTriggeringPolicy interval="6" modulate="true" />
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
index d61560e..a2bbe924 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
@@ -32,14 +32,14 @@
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Test;
 
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.parseSubFolderName;
 
 /**
  * Test for new and old style persistent storage folders generation and compatible startup of current ignite version
@@ -198,12 +198,12 @@
      * @throws IgniteCheckedException if failed.
      */
     @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
-        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsFolderResolver.DB_DEFAULT_FOLDER);
         final Set<Integer> indexes = new TreeSet<>();
-        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+        final File[] files = curFolder.listFiles(PdsFolderResolver.DB_SUBFOLDERS_NEW_STYLE_FILTER);
 
         for (File file : files) {
-            final PdsConsistentIdProcessor.FolderCandidate uid
+            final PdsFolderResolver.FolderCandidate uid
                 = parseSubFolderName(file, log);
 
             if (uid != null)
@@ -223,7 +223,7 @@
         assertDirectoryExist(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH, subDirName);
         assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_STORE_PATH, subDirName);
         assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
-        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+        assertDirectoryExist(PdsFolderResolver.DB_DEFAULT_FOLDER, subDirName);
     }
 
     /**
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
index cd97917..cab4a62 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
@@ -1524,7 +1524,8 @@
                 new GridCacheVersion(),
                 0L,
                 partId,
-                updateCntr
+                updateCntr,
+                DataEntry.EMPTY_FLAGS
             );
 
             GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database();
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
index 74464e9..0b4ddf2 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
@@ -2638,7 +2638,8 @@
                 new GridCacheVersion(),
                 0L,
                 partId,
-                updateCntr
+                updateCntr,
+                DataEntry.EMPTY_FLAGS
             );
 
             GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database();
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java
new file mode 100644
index 0000000..8aa9772
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java
@@ -0,0 +1,67 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * Entry event order.
+ * Two concurrent updates of the same entry can be ordered based on {@link CacheEntryVersion} comparsion.
+ * Greater value means that event occurs later.
+ *
+ * @see CacheConflictResolutionManager
+ * @see GridCacheVersionManager#dataCenterId(byte)
+ */
+@IgniteExperimental
+public interface CacheEntryVersion extends Comparable<CacheEntryVersion>, Serializable {
+    /**
+     * Order of the update. Value is an incremental counter value. Scope of counter is node.
+     * @return Version order.
+     */
+    public long order();
+
+    /** @return Node order on which this version was assigned. */
+    public int nodeOrder();
+
+    /**
+     * Cluster id is a value to distinguish updates in case user wants to aggregate and sort updates from several
+     * Ignite clusters. {@code clusterId} id can be set for the node using
+     * {@link GridCacheVersionManager#dataCenterId(byte)}.
+     *
+     * @return Cluster id.
+     */
+    public byte clusterId();
+
+    /** @return Topology version plus number of seconds from the start time of the first grid node. */
+    public int topologyVersion();
+
+    /**
+     * If source of the update is "local" cluster then {@code null} will be returned.
+     * If updated comes from the other cluster using {@link IgniteInternalCache#putAllConflict(Map)}
+     * then entry version for other cluster.
+     * @return Replication version.
+     * @see IgniteInternalCache#putAllConflict(Map)
+     * @see IgniteInternalCache#removeAllConflict(Map)
+     */
+    public CacheEntryVersion otherClusterVersion();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java
new file mode 100644
index 0000000..ca2d89c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java
@@ -0,0 +1,101 @@
+/*
+ * 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.cdc;
+
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * This class defines {@link CdcMain} runtime configuration.
+ * Configuration is passed to {@link CdcMain} constructor.
+ */
+@IgniteExperimental
+public class CdcConfiguration {
+    /** */
+    private static final int DFLT_LOCK_TIMEOUT = 1000;
+
+    /** */
+    private static final long DFLT_CHECK_FREQ = 1000L;
+
+    /** */
+    private static final boolean DFLT_KEEP_BINARY = true;
+
+    /** Change Data Capture consumer. */
+    private CdcConsumer consumer;
+
+    /** Keep binary flag.<br>Default value {@code true}. */
+    private boolean keepBinary = DFLT_KEEP_BINARY;
+
+    /**
+     * {@link CdcMain} acquire file lock on startup to ensure exclusive consumption.
+     * This property specifies amount of time to wait for lock acquisition.<br>
+     * Default is {@code 1000 ms}.
+     */
+    private long lockTimeout = DFLT_LOCK_TIMEOUT;
+
+    /**
+     * CDC application periodically scans {@link DataStorageConfiguration#getCdcWalPath()} folder to find new WAL segments.
+     * This frequency specify amount of time application sleeps between subsequent checks when no new files available.
+     * Default is {@code 1000 ms}.
+     */
+    private long checkFreq = DFLT_CHECK_FREQ;
+
+    /** @return CDC consumer. */
+    public CdcConsumer getConsumer() {
+        return consumer;
+    }
+
+    /** @param consumer CDC consumer. */
+    public void setConsumer(CdcConsumer consumer) {
+        this.consumer = consumer;
+    }
+
+    /** @return keep binary value. */
+    public boolean isKeepBinary() {
+        return keepBinary;
+    }
+
+    /** @param keepBinary keep binary value. */
+    public void setKeepBinary(boolean keepBinary) {
+        this.keepBinary = keepBinary;
+    }
+
+    /** @return Amount of time to wait for lock acquisition. */
+    public long getLockTimeout() {
+        return lockTimeout;
+    }
+
+    /** @param lockTimeout Amount of time to wait for lock acquisition. */
+    public void setLockTimeout(long lockTimeout) {
+        this.lockTimeout = lockTimeout;
+    }
+
+    /** @return Amount of time application sleeps between subsequent checks when no new files available. */
+    public long getCheckFrequency() {
+        return checkFreq;
+    }
+
+    /**
+     * @param checkFreq Amount of time application sleeps between subsequent checks when no new
+     *                                           files available.
+     */
+    public void setCheckFrequency(long checkFreq) {
+        this.checkFreq = checkFreq;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java
new file mode 100644
index 0000000..af119e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cdc;
+
+import java.util.Iterator;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheEntryVersion;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Consumer of WAL data change events.
+ * This consumer will receive data change events during {@link CdcMain} application invocation.
+ * The lifecycle of the consumer is the following:
+ * <ul>
+ *     <li>Start of the consumer {@link #start()}.</li>
+ *     <li>Notification of the consumer by the {@link #onEvents(Iterator)} call.</li>
+ *     <li>Stop of the consumer {@link #stop()}.</li>
+ * </ul>
+ *
+ * In case consumer implementation wants to user {@link IgniteLogger}, please, use, {@link LoggerResource} annotation:
+ * <pre>
+ * public class ChangeDataCaptureConsumer implements ChangeDataCaptureConsumer {
+ *     &#64;LoggerResource
+ *     private IgniteLogger log;
+ *
+ *     ...
+ * }
+ * </pre>
+ *
+ * Note, consumption of the {@link CdcEvent} will be started from the last saved offset.
+ * The offset of consumptions is saved on the disk every time {@link #onEvents(Iterator)} returns {@code true}.
+ *
+ * @see CdcMain
+ * @see CdcEvent
+ * @see CacheEntryVersion
+ */
+@IgniteExperimental
+public interface CdcConsumer {
+    /**
+     * Starts the consumer.
+     */
+    public void start();
+
+    /**
+     * Handles entry changes events.
+     * If this method return {@code true} then current offset will be stored
+     * and ongoing notifications after CDC application fail/restart will be started from it.
+     *
+     * @param events Entry change events.
+     * @return {@code True} if current offset should be saved on the disk
+     * to continue from it in case any failures or restart.
+     */
+    public boolean onEvents(Iterator<CdcEvent> events);
+
+    /**
+     * Stops the consumer.
+     * This methods can be invoked only after {@link #start()}.
+     */
+    public void stop();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java
new file mode 100644
index 0000000..2f8f7a5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java
@@ -0,0 +1,79 @@
+/*
+ * 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.cdc;
+
+import java.io.Serializable;
+import org.apache.ignite.cache.CacheEntryVersion;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.lang.IgniteExperimental;
+import org.apache.ignite.spi.systemview.view.CacheView;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Event of single entry change.
+ * Instance presents new value of modified entry.
+ *
+ * @see CdcMain
+ * @see CdcConsumer
+ */
+@IgniteExperimental
+public interface CdcEvent extends Serializable {
+    /**
+     * @return Key for the changed entry.
+     */
+    public Object key();
+
+    /**
+     * @return Value for the changed entry or {@code null} in case of entry removal.
+     */
+    @Nullable public Object value();
+
+    /**
+     * @return {@code True} if event fired on primary node for partition containing this entry.
+     * @see <a href="
+     * https://ignite.apache.org/docs/latest/configuring-caches/configuring-backups#configuring-partition-backups">
+     * Configuring partition backups.</a>
+     */
+    public boolean primary();
+
+    /**
+     * Ignite split dataset into smaller chunks to distribute them across the cluster.
+     * {@link CdcConsumer} implementations can use {@link #partition()} to split changes processing
+     * in the same way as it done for the cache.
+     *
+     * @return Partition number.
+     * @see Affinity#partition(Object)
+     * @see Affinity#partitions()
+     * @see <a href="https://ignite.apache.org/docs/latest/data-modeling/data-partitioning">Data partitioning</a>
+     * @see <a href="https://ignite.apache.org/docs/latest/data-modeling/affinity-collocation">Affinity collocation</a>
+     */
+    public int partition();
+
+    /**
+     * @return Version of the entry.
+     */
+    public CacheEntryVersion version();
+
+    /**
+     * @return Cache ID.
+     * @see org.apache.ignite.internal.util.typedef.internal.CU#cacheId(String)
+     * @see CacheView#cacheId()
+     */
+    public int cacheId();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java
new file mode 100644
index 0000000..b3bc583
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cdc;
+
+import java.net.URL;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
+import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteExperimental;
+
+import static org.apache.ignite.internal.IgniteComponentType.SPRING;
+
+/**
+ * Utility class to load {@link CdcMain} from Spring XML configuration.
+ */
+@IgniteExperimental
+public class CdcLoader {
+    /**
+     * Loads {@link CdcMain} from XML configuration file.
+     *
+     * @param springXmlPath Path to XML configuration file.
+     * @return {@code ChangeDataCapture} instance.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static CdcMain loadCdc(String springXmlPath) throws IgniteCheckedException {
+        URL cfgUrl = U.resolveSpringUrl(springXmlPath);
+
+        IgniteSpringHelper spring = SPRING.create(false);
+
+        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> cfgTuple =
+            spring.loadConfigurations(cfgUrl);
+
+        if (cfgTuple.get1().size() > 1) {
+            throw new IgniteCheckedException(
+                "Exact 1 IgniteConfiguration should be defined. Found " + cfgTuple.get1().size()
+            );
+        }
+
+        IgniteBiTuple<Collection<CdcConfiguration>, ? extends GridSpringResourceContext> cdcCfgs =
+            spring.loadConfigurations(cfgUrl, CdcConfiguration.class);
+
+        if (cdcCfgs.get1().size() > 1) {
+            throw new IgniteCheckedException(
+                "Exact 1 CaptureDataChangeConfiguration configuration should be defined. " +
+                    "Found " + cdcCfgs.get1().size()
+            );
+        }
+
+        return new CdcMain(
+            cfgTuple.get1().iterator().next(),
+            cfgTuple.get2(),
+            cdcCfgs.get1().iterator().next()
+        );
+    }
+}
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 594e307..e62707e 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
@@ -27,6 +27,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.lang.IgniteExperimental;
 import org.apache.ignite.mxbean.MetricsMxBean;
 import org.jetbrains.annotations.Nullable;
 
@@ -164,6 +165,9 @@
     /** Default wal archive directory. */
     public static final String DFLT_WAL_ARCHIVE_PATH = "db/wal/archive";
 
+    /** Default change data capture directory. */
+    public static final String DFLT_WAL_CDC_PATH = "db/wal/cdc";
+
     /** Default path (relative to working directory) of binary metadata folder */
     public static final String DFLT_BINARY_METADATA_PATH = "db/binary_meta";
 
@@ -244,6 +248,14 @@
     /** WAL archive path. */
     private String walArchivePath = DFLT_WAL_ARCHIVE_PATH;
 
+    /** Change Data Capture path. */
+    @IgniteExperimental
+    private String cdcWalPath = DFLT_WAL_CDC_PATH;
+
+    /** Change Data Capture enabled flag. */
+    @IgniteExperimental
+    private boolean cdcEnabled;
+
     /** Metrics enabled flag. */
     private boolean metricsEnabled = DFLT_METRICS_ENABLED;
 
@@ -291,6 +303,10 @@
      */
     private long walAutoArchiveAfterInactivity = -1;
 
+    /** Time interval (in milliseconds) for force archiving of incompletely WAL segment. */
+    @IgniteExperimental
+    private long walForceArchiveTimeout = -1;
+
     /**
      * If true, threads that generate dirty pages too fast during ongoing checkpoint will be throttled.
      */
@@ -729,6 +745,54 @@
     }
 
     /**
+     * Gets a path to the CDC directory.
+     * If this path is relative, it will be resolved relatively to Ignite work directory.
+     *
+     * @return CDC directory.
+     */
+    @IgniteExperimental
+    public String getCdcWalPath() {
+        return cdcWalPath;
+    }
+
+    /**
+     * Sets a path for the CDC directory.
+     * Hard link to every WAL Archive segment will be created in it for CDC processing purpose.
+     *
+     * @param cdcWalPath CDC directory.
+     * @return {@code this} for chaining.
+     */
+    @IgniteExperimental
+    public DataStorageConfiguration setCdcWalPath(String cdcWalPath) {
+        this.cdcWalPath = cdcWalPath;
+
+        return this;
+    }
+
+    /**
+     * Sets flag indicating whether CDC enabled.
+     *
+     * @param cdcEnabled CDC enabled flag.
+     */
+    @IgniteExperimental
+    public DataStorageConfiguration setCdcEnabled(boolean cdcEnabled) {
+        this.cdcEnabled = cdcEnabled;
+
+        return this;
+    }
+
+    /**
+     * Gets flag indicating whether CDC is enabled.
+     * Default value is {@code false}.
+     *
+     * @return Metrics enabled flag.
+     */
+    @IgniteExperimental
+    public boolean isCdcEnabled() {
+        return cdcEnabled;
+    }
+
+    /**
      * Gets flag indicating whether persistence metrics collection is enabled.
      * Default value is {@link #DFLT_METRICS_ENABLED}.
      *
@@ -1022,6 +1086,27 @@
     }
 
     /**
+     * @param walForceArchiveTimeout time in millis to run auto archiving segment (even if incomplete) after last
+     * record logging.<br> Positive value enables incomplete segment archiving after timeout (inactivity).<br> Zero or
+     * negative  value disables auto archiving.
+     * @return current configuration instance for chaining
+     */
+    @IgniteExperimental
+    public DataStorageConfiguration setWalForceArchiveTimeout(long walForceArchiveTimeout) {
+        this.walForceArchiveTimeout = walForceArchiveTimeout;
+
+        return this;
+    }
+
+    /**
+     * @return time in millis to run auto archiving WAL segment (even if incomplete) after last record log
+     */
+    @IgniteExperimental
+    public long getWalForceArchiveTimeout() {
+        return walForceArchiveTimeout;
+    }
+
+    /**
      * This property defines order of writing pages to disk storage during checkpoint.
      *
      * @return Checkpoint write order.
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 a370b4f..e206ce7 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
@@ -347,7 +347,7 @@
     public static final String SITE = "ignite.apache.org";
 
     /** System line separator. */
-    private static final String NL = U.nl();
+    public static final String NL = U.nl();
 
     /** System megabyte. */
     private static final int MEGABYTE = 1024 * 1024;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index f1e5b69..a9a6e95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -22,8 +22,6 @@
 import java.io.InputStream;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.management.ManagementFactory;
-import java.lang.reflect.Constructor;
-import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -109,8 +107,6 @@
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.logger.LoggerNodeIdAware;
-import org.apache.ignite.logger.java.JavaLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
@@ -2301,7 +2297,7 @@
             if (!F.isEmpty(predefineConsistentId))
                 myCfg.setConsistentId(predefineConsistentId);
 
-            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId, workDir);
+            IgniteLogger cfgLog = U.initLogger(cfg.getGridLogger(), null, nodeId, workDir);
 
             assert cfgLog != null;
 
@@ -2558,92 +2554,6 @@
         }
 
         /**
-         * @param cfgLog Configured logger.
-         * @param nodeId Local node ID.
-         * @param workDir Work directory.
-         * @return Initialized logger.
-         * @throws IgniteCheckedException If failed.
-         */
-        @SuppressWarnings("ErrorNotRethrown")
-        private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir)
-            throws IgniteCheckedException {
-            try {
-                Exception log4jInitErr = null;
-
-                if (cfgLog == null) {
-                    Class<?> log4jCls;
-
-                    try {
-                        log4jCls = Class.forName("org.apache.ignite.logger.log4j.Log4JLogger");
-                    }
-                    catch (ClassNotFoundException | NoClassDefFoundError ignored) {
-                        log4jCls = null;
-                    }
-
-                    if (log4jCls != null) {
-                        try {
-                            URL url = U.resolveIgniteUrl("config/ignite-log4j.xml");
-
-                            if (url == null) {
-                                File cfgFile = new File("config/ignite-log4j.xml");
-
-                                if (!cfgFile.exists())
-                                    cfgFile = new File("../config/ignite-log4j.xml");
-
-                                if (cfgFile.exists()) {
-                                    try {
-                                        url = cfgFile.toURI().toURL();
-                                    }
-                                    catch (MalformedURLException ignore) {
-                                        // No-op.
-                                    }
-                                }
-                            }
-
-                            if (url != null) {
-                                boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null);
-
-                                if (configured)
-                                    url = null;
-                            }
-
-                            if (url != null) {
-                                Constructor<?> ctor = log4jCls.getConstructor(URL.class);
-
-                                cfgLog = (IgniteLogger)ctor.newInstance(url);
-                            }
-                            else
-                                cfgLog = (IgniteLogger)log4jCls.newInstance();
-                        }
-                        catch (Exception e) {
-                            log4jInitErr = e;
-                        }
-                    }
-
-                    if (log4jCls == null || log4jInitErr != null)
-                        cfgLog = new JavaLogger();
-                }
-
-                // Special handling for Java logger which requires work directory.
-                if (cfgLog instanceof JavaLogger)
-                    ((JavaLogger)cfgLog).setWorkDirectory(workDir);
-
-                // Set node IDs for all file appenders.
-                if (cfgLog instanceof LoggerNodeIdAware)
-                    ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId);
-
-                if (log4jInitErr != null)
-                    U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): "
-                        + log4jInitErr.getCause());
-
-                return cfgLog;
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to create logger.", e);
-            }
-        }
-
-        /**
          * Creates utility system cache configuration.
          *
          * @return Utility system cache configuration.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java
new file mode 100644
index 0000000..0ac8b57
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java
@@ -0,0 +1,114 @@
+/*
+ * 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.cdc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Iterator;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cdc.CdcConsumer;
+import org.apache.ignite.cdc.CdcEvent;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer.POINTER_SIZE;
+
+/**
+ * Change Data Capture Consumer state.
+ *
+ * Each time {@link CdcConsumer#onEvents(Iterator)} returns {@code true}
+ * current offset in WAL segment saved to file.
+ * This allows to the {@link CdcConsumer} to continue consumption of the {@link CdcEvent}
+ * from the last saved offset in case of fail or restart.
+ *
+ * @see CdcConsumer#onEvents(Iterator)
+ * @see CdcMain
+ */
+public class CdcConsumerState {
+    /** */
+    public static final String STATE_FILE_NAME = "cdc-state" + FILE_SUFFIX;
+
+    /** State file. */
+    private final Path state;
+
+    /** Temp state file. */
+    private final Path tmp;
+
+    /**
+     * @param stateDir State directory.
+     */
+    public CdcConsumerState(Path stateDir) {
+        state = stateDir.resolve(STATE_FILE_NAME);
+        tmp = stateDir.resolve(STATE_FILE_NAME + TMP_SUFFIX);
+    }
+
+    /**
+     * Saves state to file.
+     * @param ptr WAL pointer.
+     */
+    public void save(WALPointer ptr) throws IOException {
+        ByteBuffer buf = ByteBuffer.allocate(POINTER_SIZE);
+
+        buf.putLong(ptr.index());
+        buf.putInt(ptr.fileOffset());
+        buf.putInt(ptr.length());
+        buf.flip();
+
+        try (FileChannel ch = FileChannel.open(tmp, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            ch.write(buf);
+
+            ch.force(true);
+        }
+
+        Files.move(tmp, state, ATOMIC_MOVE, REPLACE_EXISTING);
+    }
+
+    /**
+     * Loads CDC state from file.
+     * @return Saved state.
+     */
+    public WALPointer load() {
+        if (!Files.exists(state))
+            return null;
+
+        try (FileChannel ch = FileChannel.open(state, StandardOpenOption.READ)) {
+            ByteBuffer buf = ByteBuffer.allocate(POINTER_SIZE);
+
+            ch.read(buf);
+
+            buf.flip();
+
+            long idx = buf.getLong();
+            int offset = buf.getInt();
+            int length = buf.getInt();
+
+            return new WALPointer(idx, offset, length);
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to read state [file=" + state + ']', e);
+        }
+
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java
new file mode 100644
index 0000000..a12aa0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java
@@ -0,0 +1,106 @@
+/*
+ * 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.cdc;
+
+import org.apache.ignite.cache.CacheEntryVersion;
+import org.apache.ignite.cdc.CdcConsumer;
+import org.apache.ignite.cdc.CdcEvent;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Event of single entry change.
+ * Instance presents new value of modified entry.
+ *
+ * @see CdcMain
+ * @see CdcConsumer
+ */
+public class CdcEventImpl implements CdcEvent {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Key. */
+    private final Object key;
+
+    /** Value. */
+    private final Object val;
+
+    /** {@code True} if changes made on primary node. */
+    private final boolean primary;
+
+    /** Partition. */
+    private final int part;
+
+    /** Order of the entry change. */
+    private final CacheEntryVersion ord;
+
+    /** Cache id. */
+    private final int cacheId;
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param primary {@code True} if changes made on primary node.
+     * @param part Partition.
+     * @param ord Order of the entry change.
+     * @param cacheId Cache id.
+     */
+    public CdcEventImpl(Object key, Object val, boolean primary, int part,
+        CacheEntryVersion ord, int cacheId) {
+        this.key = key;
+        this.val = val;
+        this.primary = primary;
+        this.part = part;
+        this.ord = ord;
+        this.cacheId = cacheId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value() {
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean primary() {
+        return primary;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return part;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheEntryVersion version() {
+        return ord;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int cacheId() {
+        return cacheId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CdcEventImpl.class, this);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java
new file mode 100644
index 0000000..093e235
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cdc;
+
+import java.lang.management.ManagementFactory;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder;
+
+/**
+ * Lock file holder for Change Data Capture application.
+ *
+ * @see CdcMain
+ * @see CdcConsumerState
+ */
+public class CdcFileLockHolder extends FileLockHolder {
+    /** Consumer ID. */
+    private final String consumerId;
+
+    /**
+     * @param rootDir Root directory for lock file.
+     * @param log Log.
+     */
+    public CdcFileLockHolder(String rootDir, String consumerId, IgniteLogger log) {
+        super(rootDir, log);
+
+        this.consumerId = consumerId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String lockInfo() {
+        return "[consumerId=" + consumerId + ", proc=" + ManagementFactory.getRuntimeMXBean().getName() + ']';
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String warningMessage(String lockInfo) {
+        return "Failed to acquire file lock. Will try again in 1s " +
+            "[proc=" + ManagementFactory.getRuntimeMXBean().getName() + ", holder=" + lockInfo +
+            ", path=" + lockPath() + ']';
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java
new file mode 100644
index 0000000..d2b7bd3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java
@@ -0,0 +1,556 @@
+/*
+ * 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.cdc;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Stream;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.CdcConfiguration;
+import org.apache.ignite.cdc.CdcConsumer;
+import org.apache.ignite.cdc.CdcEvent;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridLoggerProxy;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
+import org.apache.ignite.internal.processors.resource.GridResourceIoc;
+import org.apache.ignite.internal.processors.resource.GridResourceLoggerInjector;
+import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.resources.SpringApplicationContextResource;
+import org.apache.ignite.resources.SpringResource;
+import org.apache.ignite.startup.cmdline.CdcCommandLineStartup;
+
+import static org.apache.ignite.internal.IgniteKernal.NL;
+import static org.apache.ignite.internal.IgniteKernal.SITE;
+import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR;
+import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER;
+
+/**
+ * Change Data Capture (CDC) application.
+ * The application runs independently of Ignite node process and provides the ability
+ * for the {@link CdcConsumer} to consume events({@link CdcEvent}) from WAL segments.
+ * The user should provide {@link CdcConsumer} implementation with custom consumption logic.
+ *
+ * Ignite node should be explicitly configured for using {@link CdcMain}.
+ * <ol>
+ *     <li>Set {@link DataStorageConfiguration#setCdcEnabled(boolean)} to true.</li>
+ *     <li>Optional: Set {@link DataStorageConfiguration#setCdcWalPath(String)} to path to the directory
+ *     to store WAL segments for CDC.</li>
+ *     <li>Optional: Set {@link DataStorageConfiguration#setWalForceArchiveTimeout(long)} to configure timeout for
+ *     force WAL rollover, so new events will be available for consumptions with the predicted time.</li>
+ * </ol>
+ *
+ * When {@link DataStorageConfiguration#getCdcWalPath()} is true then Ignite node on each WAL segment
+ * rollover creates hard link to archive WAL segment in
+ * {@link DataStorageConfiguration#getCdcWalPath()} directory. {@link CdcMain} application takes
+ * segment file and consumes events from it.
+ * After successful consumption (see {@link CdcConsumer#onEvents(Iterator)}) WAL segment will be deleted
+ * from directory.
+ *
+ * Several Ignite nodes can be started on the same host.
+ * If your deployment done with custom consistent id then you should specify it via
+ * {@link IgniteConfiguration#setConsistentId(Serializable)} in provided {@link IgniteConfiguration}.
+ *
+ * Application works as follows:
+ * <ol>
+ *     <li>Searches node work directory based on provided {@link IgniteConfiguration}.</li>
+ *     <li>Awaits for the creation of CDC directory if it not exists.</li>
+ *     <li>Acquires file lock to ensure exclusive consumption.</li>
+ *     <li>Loads state of consumption if it exists.</li>
+ *     <li>Infinitely waits for new available segment and processes it.</li>
+ * </ol>
+ *
+ * @see DataStorageConfiguration#setCdcEnabled(boolean)
+ * @see DataStorageConfiguration#setCdcWalPath(String)
+ * @see DataStorageConfiguration#setWalForceArchiveTimeout(long)
+ * @see CdcCommandLineStartup
+ * @see CdcConsumer
+ * @see DataStorageConfiguration#DFLT_WAL_CDC_PATH
+ */
+public class CdcMain implements Runnable {
+    /** */
+    public static final String ERR_MSG = "Persistence disabled. Capture Data Change can't run!";
+
+    /** State dir. */
+    public static final String STATE_DIR = "state";
+
+    /** Ignite configuration. */
+    private final IgniteConfiguration igniteCfg;
+
+    /** Spring resource context. */
+    private final GridSpringResourceContext ctx;
+
+    /** Change Data Capture configuration. */
+    private final CdcConfiguration cdcCfg;
+
+    /** WAL iterator factory. */
+    private final IgniteWalIteratorFactory factory;
+
+    /** Events consumer. */
+    private final WalRecordsConsumer<?, ?> consumer;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Change Data Capture directory. */
+    private Path cdcDir;
+
+    /** Binary meta directory. */
+    private File binaryMeta;
+
+    /** Marshaller directory. */
+    private File marshaller;
+
+    /** Change Data Capture state. */
+    private CdcConsumerState state;
+
+    /** Save state to start from. */
+    private WALPointer initState;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** Already processed segments. */
+    private final Set<Path> processedSegments = new HashSet<>();
+
+    /**
+     * @param cfg Ignite configuration.
+     * @param ctx Spring resource context.
+     * @param cdcCfg Change Data Capture configuration.
+     */
+    public CdcMain(
+        IgniteConfiguration cfg,
+        GridSpringResourceContext ctx,
+        CdcConfiguration cdcCfg) {
+        igniteCfg = new IgniteConfiguration(cfg);
+        this.ctx = ctx;
+        this.cdcCfg = cdcCfg;
+
+        try {
+            U.initWorkDir(igniteCfg);
+
+            log = U.initLogger(igniteCfg, "ignite-cdc");
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+
+        consumer = new WalRecordsConsumer<>(cdcCfg.getConsumer(), log);
+
+        factory = new IgniteWalIteratorFactory(log);
+    }
+
+    /** Runs Change Data Capture. */
+    @Override public void run() {
+        synchronized (this) {
+            if (stopped)
+                return;
+        }
+
+        try {
+            runX();
+        }
+        catch (Throwable e) {
+            log.error("Cdc error", e);
+
+            throw new IgniteException(e);
+        }
+    }
+
+    /** Runs Change Data Capture application with possible exception. */
+    public void runX() throws Exception {
+        ackAsciiLogo();
+
+        if (!CU.isPersistenceEnabled(igniteCfg)) {
+            log.error(ERR_MSG);
+
+            throw new IllegalArgumentException(ERR_MSG);
+        }
+
+        PdsFolderSettings<CdcFileLockHolder> settings =
+            new PdsFolderResolver<>(igniteCfg, log, null, this::tryLock).resolve();
+
+        if (settings == null) {
+            throw new IgniteException("Can't find folder to read WAL segments from based on provided configuration! " +
+                "[workDir=" + igniteCfg.getWorkDirectory() + ", consistentId=" + igniteCfg.getConsistentId() + ']');
+        }
+
+        CdcFileLockHolder lock = settings.getLockedFileLockHolder();
+
+        if (lock == null) {
+            File consIdDir = new File(settings.persistentStoreRootPath(), settings.folderName());
+
+            lock = tryLock(consIdDir);
+
+            if (lock == null) {
+                throw new IgniteException(
+                    "Can't acquire lock for Change Data Capture folder [dir=" + consIdDir.getAbsolutePath() + ']'
+                );
+            }
+        }
+
+        try {
+            String consIdDir = cdcDir.getName(cdcDir.getNameCount() - 1).toString();
+
+            Files.createDirectories(cdcDir.resolve(STATE_DIR));
+
+            binaryMeta = CacheObjectBinaryProcessorImpl.binaryWorkDir(igniteCfg.getWorkDirectory(), consIdDir);
+
+            marshaller = MarshallerContextImpl.mappingFileStoreWorkDir(igniteCfg.getWorkDirectory());
+
+            if (log.isInfoEnabled()) {
+                log.info("Change Data Capture [dir=" + cdcDir + ']');
+                log.info("Ignite node Binary meta [dir=" + binaryMeta + ']');
+                log.info("Ignite node Marshaller [dir=" + marshaller + ']');
+            }
+
+            injectResources(consumer.consumer());
+
+            state = new CdcConsumerState(cdcDir.resolve(STATE_DIR));
+
+            initState = state.load();
+
+            if (initState != null && log.isInfoEnabled())
+                log.info("Initial state loaded [state=" + initState + ']');
+
+            consumer.start();
+
+            try {
+                consumeWalSegmentsUntilStopped();
+            }
+            finally {
+                consumer.stop();
+
+                if (log.isInfoEnabled())
+                    log.info("Ignite Change Data Capture Application stopped.");
+            }
+        }
+        finally {
+            U.closeQuiet(lock);
+        }
+    }
+
+    /** Waits and consumes new WAL segments until stopped. */
+    public void consumeWalSegmentsUntilStopped() {
+        try {
+            Set<Path> seen = new HashSet<>();
+
+            AtomicLong lastSgmnt = new AtomicLong(-1);
+
+            while (!stopped) {
+                try (Stream<Path> cdcFiles = Files.walk(cdcDir, 1)) {
+                    Set<Path> exists = new HashSet<>();
+
+                    cdcFiles
+                        .peek(exists::add) // Store files that exists in cdc dir.
+                        // Need unseen WAL segments only.
+                        .filter(p -> WAL_SEGMENT_FILE_FILTER.accept(p.toFile()) && !seen.contains(p))
+                        .peek(seen::add) // Adds to seen.
+                        .sorted(Comparator.comparingLong(this::segmentIndex)) // Sort by segment index.
+                        .peek(p -> {
+                            long nextSgmnt = segmentIndex(p);
+
+                            assert lastSgmnt.get() == -1 || nextSgmnt - lastSgmnt.get() == 1;
+
+                            lastSgmnt.set(nextSgmnt);
+                        })
+                        .forEach(this::consumeSegment); // Consuming segments.
+
+                    seen.removeIf(p -> !exists.contains(p)); // Clean up seen set.
+                }
+
+                if (!stopped)
+                    U.sleep(cdcCfg.getCheckFrequency());
+            }
+        }
+        catch (IOException | IgniteInterruptedCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** Reads all available records from segment. */
+    private void consumeSegment(Path segment) {
+        if (log.isInfoEnabled())
+            log.info("Processing WAL segment [segment=" + segment + ']');
+
+        IgniteWalIteratorFactory.IteratorParametersBuilder builder =
+            new IgniteWalIteratorFactory.IteratorParametersBuilder()
+                .log(log)
+                .binaryMetadataFileStoreDir(binaryMeta)
+                .marshallerMappingFileStoreDir(marshaller)
+                .keepBinary(cdcCfg.isKeepBinary())
+                .filesOrDirs(segment.toFile())
+                .addFilter((type, ptr) -> type == DATA_RECORD_V2);
+
+        if (initState != null) {
+            long segmentIdx = segmentIndex(segment);
+
+            if (segmentIdx > initState.index()) {
+                throw new IgniteException("Found segment greater then saved state. Some events are missed. Exiting! " +
+                    "[state=" + initState + ", segment=" + segmentIdx + ']');
+            }
+
+            if (segmentIdx < initState.index()) {
+                if (log.isInfoEnabled()) {
+                    log.info("Already processed segment found. Skipping and deleting the file [segment=" +
+                        segmentIdx + ", state=" + initState.index() + ']');
+                }
+
+                // WAL segment is a hard link to a segment file in the special Change Data Capture folder.
+                // So, we can safely delete it after processing.
+                try {
+                    Files.delete(segment);
+
+                    return;
+                }
+                catch (IOException e) {
+                    throw new IgniteException(e);
+                }
+            }
+
+            builder.from(initState);
+
+            initState = null;
+        }
+
+        try (WALIterator it = factory.iterator(builder)) {
+            boolean interrupted = Thread.interrupted();
+
+            while (it.hasNext() && !interrupted) {
+                Iterator<DataRecord> iter = F.iterator(it.iterator(), t -> (DataRecord)t.get2(), true);
+
+                boolean commit = consumer.onRecords(iter);
+
+                if (commit) {
+                    assert it.lastRead().isPresent();
+
+                    state.save(it.lastRead().get());
+
+                    // Can delete after new file state save.
+                    if (!processedSegments.isEmpty()) {
+                        // WAL segment is a hard link to a segment file in a specifal Change Data Capture folder.
+                        // So we can safely delete it after success processing.
+                        for (Path processedSegment : processedSegments) {
+                            // Can't delete current segment, because state points to it.
+                            if (processedSegment.equals(segment))
+                                continue;
+
+                            Files.delete(processedSegment);
+                        }
+
+                        processedSegments.clear();
+                    }
+                }
+
+                interrupted = Thread.interrupted();
+            }
+
+            if (interrupted)
+                throw new IgniteException("Change Data Capture Application interrupted");
+
+            processedSegments.add(segment);
+        } catch (IgniteCheckedException | IOException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Try locks Change Data Capture directory.
+     *
+     * @param dbStoreDirWithSubdirectory Root PDS directory.
+     * @return Lock or null if lock failed.
+     */
+    private CdcFileLockHolder tryLock(File dbStoreDirWithSubdirectory) {
+        if (!dbStoreDirWithSubdirectory.exists()) {
+            log.warning("DB store directory not exists [dir=" + dbStoreDirWithSubdirectory + ']');
+
+            return null;
+        }
+
+        File cdcRoot = new File(igniteCfg.getDataStorageConfiguration().getCdcWalPath());
+
+        if (!cdcRoot.isAbsolute()) {
+            cdcRoot = new File(
+                igniteCfg.getWorkDirectory(),
+                igniteCfg.getDataStorageConfiguration().getCdcWalPath()
+            );
+        }
+
+        if (!cdcRoot.exists()) {
+            log.warning("CDC root directory not exists. Should be created by Ignite Node. " +
+                "Is Change Data Capture enabled in IgniteConfiguration? [dir=" + cdcRoot + ']');
+
+            return null;
+        }
+
+        Path cdcDir = Paths.get(cdcRoot.getAbsolutePath(), dbStoreDirWithSubdirectory.getName());
+
+        if (!Files.exists(cdcDir)) {
+            log.warning("CDC directory not exists. Should be created by Ignite Node. " +
+                "Is Change Data Capture enabled in IgniteConfiguration? [dir=" + cdcDir + ']');
+
+            return null;
+        }
+
+        this.cdcDir = cdcDir;
+
+        CdcFileLockHolder lock = new CdcFileLockHolder(cdcDir.toString(), "cdc.lock", log);
+
+        try {
+            lock.tryLock(cdcCfg.getLockTimeout());
+
+            return lock;
+        }
+        catch (IgniteCheckedException e) {
+            U.closeQuiet(lock);
+
+            if (log.isInfoEnabled()) {
+                log.info("Unable to acquire lock to lock CDC folder [dir=" + cdcRoot + "]" + NL +
+                    "Reason: " + e.getMessage());
+            }
+
+            return null;
+        }
+    }
+
+    /**
+     * @param segment WAL segment file.
+     * @return Segment index.
+     */
+    public long segmentIndex(Path segment) {
+        String fn = segment.getFileName().toString();
+
+        return Long.parseLong(fn.substring(0, fn.indexOf('.')));
+    }
+
+    /** Stops the application. */
+    public void stop() {
+        synchronized (this) {
+            if (log.isInfoEnabled())
+                log.info("Stopping Change Data Capture service instance");
+
+            stopped = true;
+        }
+    }
+
+    /** */
+    private void injectResources(CdcConsumer dataConsumer) throws IgniteCheckedException {
+        GridResourceIoc ioc = new GridResourceIoc();
+
+        ioc.inject(
+            dataConsumer,
+            LoggerResource.class,
+            new GridResourceLoggerInjector(log),
+            null,
+            null
+        );
+
+        if (ctx != null) {
+            ioc.inject(
+                dataConsumer,
+                SpringResource.class,
+                ctx.springBeanInjector(),
+                null,
+                null
+            );
+
+            ioc.inject(
+                dataConsumer,
+                SpringApplicationContextResource.class,
+                ctx.springContextInjector(),
+                null,
+                null
+            );
+        }
+    }
+
+    /** */
+    private void ackAsciiLogo() {
+        String ver = "ver. " + ACK_VER_STR;
+
+        if (log.isInfoEnabled()) {
+            log.info(NL + NL +
+                ">>>    __________  ________________    ________  _____" + NL +
+                ">>>   /  _/ ___/ |/ /  _/_  __/ __/   / ___/ _ \\/ ___/" + NL +
+                ">>>  _/ // (7 7    // /  / / / _/    / /__/ // / /__  " + NL +
+                ">>> /___/\\___/_/|_/___/ /_/ /___/    \\___/____/\\___/  " + NL +
+                ">>> " + NL +
+                ">>> " + ver + NL +
+                ">>> " + COPYRIGHT + NL +
+                ">>> " + NL +
+                ">>> Ignite documentation: " + "http://" + SITE + NL +
+                ">>> Consumer: " + U.toStringSafe(consumer.consumer()) + NL +
+                ">>> ConsistentId: " + igniteCfg.getConsistentId() + NL
+            );
+        }
+
+        if (log.isQuiet()) {
+            U.quiet(false,
+                "   __________  ________________    ________  _____",
+                "  /  _/ ___/ |/ /  _/_  __/ __/   / ___/ _ \\/ ___/",
+                " _/ // (7 7    // /  / / / _/    / /__/ // / /__  ",
+                "/___/\\___/_/|_/___/ /_/ /___/    \\___/____/\\___/  ",
+                "",
+                ver,
+                COPYRIGHT,
+                "",
+                "Ignite documentation: " + "http://" + SITE,
+                "Consumer: " + U.toStringSafe(consumer.consumer()),
+                "ConsistentId: " + igniteCfg.getConsistentId(),
+                "",
+                "Quiet mode.");
+
+            String fileName = log.fileName();
+
+            if (fileName != null)
+                U.quiet(false, "  ^-- Logging to file '" + fileName + '\'');
+
+            if (log instanceof GridLoggerProxy)
+                U.quiet(false, "  ^-- Logging by '" + ((GridLoggerProxy)log).getLoggerInfo() + '\'');
+
+            U.quiet(false,
+                "  ^-- To see **FULL** console log here add -DIGNITE_QUIET=false or \"-v\" to ignite-cdc.{sh|bat}",
+                "");
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java
new file mode 100644
index 0000000..38c2938
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java
@@ -0,0 +1,174 @@
+/*
+ * 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.cdc;
+
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.CdcConsumer;
+import org.apache.ignite.cdc.CdcEvent;
+import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
+import org.apache.ignite.internal.pagemem.wal.record.UnwrappedDataEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgnitePredicate;
+
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
+
+/**
+ * Transform {@link DataEntry} to {@link CdcEvent} and sends it to {@link CdcConsumer}.
+ *
+ * @see CdcMain
+ * @see CdcConsumer
+ */
+public class WalRecordsConsumer<K, V> {
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Data change events consumer. */
+    private final CdcConsumer consumer;
+
+    /** Operations types we interested in. */
+    private static final EnumSet<GridCacheOperation> OPERATIONS_TYPES = EnumSet.of(CREATE, UPDATE, DELETE, TRANSFORM);
+
+    /** Operations filter. */
+    private static final IgnitePredicate<? super DataEntry> OPERATIONS_FILTER = e -> {
+        if (!(e instanceof UnwrappedDataEntry))
+            throw new IllegalStateException("Unexpected data entry [type=" + e.getClass().getName() + ']');
+
+        if ((e.flags() & DataEntry.PRELOAD_FLAG) != 0 ||
+            (e.flags() & DataEntry.FROM_STORE_FLAG) != 0)
+            return false;
+
+        return OPERATIONS_TYPES.contains(e.op());
+    };
+
+    /**
+     * @param consumer User provided CDC consumer.
+     * @param log Logger.
+     */
+    public WalRecordsConsumer(CdcConsumer consumer, IgniteLogger log) {
+        this.consumer = consumer;
+        this.log = log;
+    }
+
+    /**
+     * Handles record from the WAL.
+     * If this method return {@code true} then current offset in WAL will be stored and WAL iteration will be
+     * started from it on CDC application fail/restart.
+     *
+     * @param recs WAL records iterator.
+     * @return {@code True} if current offset in WAL should be commited.
+     */
+    public boolean onRecords(Iterator<DataRecord> recs) {
+        Iterator<CdcEvent> evts = new Iterator<CdcEvent>() {
+            /** */
+            private Iterator<CdcEvent> entries;
+
+            @Override public boolean hasNext() {
+                advance();
+
+                return hasCurrent();
+            }
+
+            @Override public CdcEvent next() {
+                advance();
+
+                if (!hasCurrent())
+                    throw new NoSuchElementException();
+
+                return entries.next();
+            }
+
+            private void advance() {
+                if (hasCurrent())
+                    return;
+
+                while (recs.hasNext()) {
+                    entries =
+                        F.iterator(recs.next().writeEntries().iterator(), this::transform, true, OPERATIONS_FILTER);
+
+                    if (entries.hasNext())
+                        break;
+
+                    entries = null;
+                }
+            }
+
+            private boolean hasCurrent() {
+                return entries != null && entries.hasNext();
+            }
+
+            /** */
+            private CdcEvent transform(DataEntry e) {
+                UnwrappedDataEntry ue = (UnwrappedDataEntry)e;
+
+                return new CdcEventImpl(
+                    ue.unwrappedKey(),
+                    ue.unwrappedValue(),
+                    (e.flags() & DataEntry.PRIMARY_FLAG) != 0,
+                    e.partitionId(),
+                    e.writeVersion(),
+                    e.cacheId()
+                );
+            }
+        };
+
+        return consumer.onEvents(evts);
+    }
+
+    /**
+     * Starts the consumer.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void start() throws IgniteCheckedException {
+        consumer.start();
+
+        if (log.isDebugEnabled())
+            log.debug("WalRecordsConsumer started [consumer=" + consumer.getClass() + ']');
+    }
+
+    /**
+     * Stops the consumer.
+     * This methods can be invoked only after {@link #start()}.
+     */
+    public void stop() {
+        consumer.stop();
+
+        if (log.isInfoEnabled())
+            log.info("WalRecordsConsumer stopped [consumer=" + consumer.getClass() + ']');
+    }
+
+    /** @return Change Data Capture Consumer. */
+    public CdcConsumer consumer() {
+        return consumer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(WalRecordsConsumer.class, this);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
index 9376bb0..096b1e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
@@ -29,6 +29,18 @@
  * Represents Data Entry ({@link #key}, {@link #val value}) pair update {@link #op operation} in WAL log.
  */
 public class DataEntry {
+    /** Empty flags. */
+    public static final byte EMPTY_FLAGS = 0;
+
+    /** */
+    public static final byte PRIMARY_FLAG = 0b00000001;
+
+    /** */
+    public static final byte PRELOAD_FLAG = 0b00000010;
+
+    /** */
+    public static final byte FROM_STORE_FLAG = 0b00000100;
+
     /** Cache ID. */
     @GridToStringInclude
     protected int cacheId;
@@ -62,6 +74,17 @@
     @GridToStringInclude
     protected long partCnt;
 
+    /**
+     * Bit flags.
+     * <ul>
+     *  <li>0 bit - primary - seted when current node is primary for entry partition.</li>
+     *  <li>1 bit - preload - seted when entry logged during preload(rebalance).</li>
+     *  <li>2 bit - fromStore - seted when entry loaded from third-party store.</li>
+     * </ul>
+     */
+    @GridToStringInclude
+    protected byte flags;
+
     /** Constructor. */
     private DataEntry() {
         // No-op, used from factory methods.
@@ -77,6 +100,7 @@
      * @param expireTime Expire time.
      * @param partId Partition ID.
      * @param partCnt Partition counter.
+     * @param flags Entry flags.
      */
     public DataEntry(
         int cacheId,
@@ -87,7 +111,8 @@
         GridCacheVersion writeVer,
         long expireTime,
         int partId,
-        long partCnt
+        long partCnt,
+        byte flags
     ) {
         this.cacheId = cacheId;
         this.key = key;
@@ -98,6 +123,7 @@
         this.expireTime = expireTime;
         this.partId = partId;
         this.partCnt = partCnt;
+        this.flags = flags;
 
         // Only READ, CREATE, UPDATE and DELETE operations should be stored in WAL.
         assert op == GridCacheOperation.READ
@@ -107,6 +133,30 @@
     }
 
     /**
+     * @param primary {@code True} if node is primary for partition in the moment of logging.
+     * @return Flags value.
+     */
+    public static byte flags(boolean primary) {
+        return flags(primary, false, false);
+    }
+
+    /**
+     * @param primary {@code True} if node is primary for partition in the moment of logging.
+     * @param preload {@code True} if logged during preload(rebalance).
+     * @param fromStore {@code True} if logged during loading from third-party store.
+     * @return Flags value.
+     */
+    public static byte flags(boolean primary, boolean preload, boolean fromStore) {
+        byte val = EMPTY_FLAGS;
+
+        val |= primary ? PRIMARY_FLAG : EMPTY_FLAGS;
+        val |= preload ? PRELOAD_FLAG : EMPTY_FLAGS;
+        val |= fromStore ? FROM_STORE_FLAG : EMPTY_FLAGS;
+
+        return val;
+    }
+
+    /**
      * @return Cache ID.
      */
     public int cacheId() {
@@ -181,6 +231,14 @@
         return expireTime;
     }
 
+    /**
+     * Entry flags.
+     * @see #flags
+     */
+    public byte flags() {
+        return flags;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataEntry.class, this);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
index ef6c3ba..2507fd4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
@@ -35,7 +35,7 @@
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.DATA_RECORD;
+        return RecordType.DATA_RECORD_V2;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
index ba2fabc..45bd54a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
@@ -60,6 +60,7 @@
      * @param expireTime Expire time.
      * @param partId Partition ID.
      * @param partCnt Partition counter.
+     * @param flags Flags.
      */
     public LazyDataEntry(
         GridCacheSharedContext cctx,
@@ -73,9 +74,10 @@
         GridCacheVersion writeVer,
         long expireTime,
         int partId,
-        long partCnt
+        long partCnt,
+        byte flags
     ) {
-        super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt);
+        super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt, flags);
 
         this.cctx = cctx;
         this.keyType = keyType;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java
index c86593f..387c147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java
@@ -56,7 +56,7 @@
         long partCnt,
         MvccVersion mvccVer
     ) {
-        super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt);
+        super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, EMPTY_FLAGS);
 
         this.mvccVer = mvccVer;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
index ad5e1d0..216e733 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
@@ -48,6 +48,7 @@
      * @param partCnt Partition counter.
      * @param cacheObjValCtx cache object value context for unwrapping objects.
      * @param keepBinary disable unwrapping for non primitive objects, Binary Objects would be returned instead.
+     * @param flags Flags.
      */
     public UnwrapDataEntry(
         final int cacheId,
@@ -60,8 +61,9 @@
         final int partId,
         final long partCnt,
         final CacheObjectValueContext cacheObjValCtx,
-        final boolean keepBinary) {
-        super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt);
+        final boolean keepBinary,
+        final byte flags) {
+        super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, flags);
         this.cacheObjValCtx = cacheObjValCtx;
         this.keepBinary = keepBinary;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
index 52461698..4a211f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
@@ -45,7 +45,8 @@
         /** */
         PAGE_RECORD(1, PHYSICAL),
 
-        /** */
+        /** @deprecated Use {@link #DATA_RECORD_V2} instead. */
+        @Deprecated
         DATA_RECORD(2, LOGICAL),
 
         /** Checkpoint (begin) record */
@@ -206,7 +207,11 @@
         /** Encrypted WAL-record. */
         ENCRYPTED_RECORD(52, PHYSICAL),
 
-        /** Ecnrypted data record. */
+        /**
+         * Ecnrypted data record.
+         * @deprecated Use {@link #ENCRYPTED_DATA_RECORD_V3} instead.
+         */
+        @Deprecated
         ENCRYPTED_DATA_RECORD(53, LOGICAL),
 
         /** Mvcc data record. */
@@ -239,7 +244,11 @@
         /** Encrypted WAL-record. */
         ENCRYPTED_RECORD_V2(63, PHYSICAL),
 
-        /** Ecnrypted data record. */
+        /**
+         * Ecnrypted data record.
+         * @deprecated Use {@link #ENCRYPTED_DATA_RECORD_V3} instead.
+         */
+        @Deprecated
         ENCRYPTED_DATA_RECORD_V2(64, LOGICAL),
 
         /** Master key change record containing multiple keys for single cache group. */
@@ -252,7 +261,13 @@
         PARTITION_META_PAGE_DELTA_RECORD_V3(67, PHYSICAL),
 
         /** Index meta page delta record includes encryption status data. */
-        INDEX_META_PAGE_DELTA_RECORD(68, PHYSICAL);
+        INDEX_META_PAGE_DELTA_RECORD(68, PHYSICAL),
+
+        /** Data record V2. */
+        DATA_RECORD_V2(69, LOGICAL),
+
+        /** Ecnrypted data record. */
+        ENCRYPTED_DATA_RECORD_V3(70, LOGICAL);
 
         /** Index for serialization. Should be consistent throughout all versions. */
         private final int idx;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index c8aeae5..65381dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3864,7 +3864,8 @@
                 false,
                 topVer,
                 replicate ? DR_LOAD : DR_NONE,
-                true);
+                true,
+                false);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to put cache value: " + entry, e);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index fb68256..43220f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -756,6 +756,7 @@
      * @param topVer Topology version.
      * @param drType DR type.
      * @param fromStore {@code True} if value was loaded from store.
+     * @param primary {@code True} if current node is primary for partition.
      * @return {@code True} if initial value was set.
      * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
@@ -767,9 +768,10 @@
         boolean preload,
         AffinityTopologyVersion topVer,
         GridDrType drType,
-        boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        boolean fromStore,
+        boolean primary) throws IgniteCheckedException, GridCacheEntryRemovedException {
         return initialValue(val, ver, null, null, TxState.NA, TxState.NA,
-            ttl, expireTime, preload, topVer, drType, fromStore);
+            ttl, expireTime, preload, topVer, drType, fromStore, primary);
     }
 
     /**
@@ -787,6 +789,7 @@
      * @param topVer Topology version.
      * @param drType DR type.
      * @param fromStore {@code True} if value was loaded from store.
+     * @param primary {@code True} if current node is primary for partition.
      * @return {@code True} if initial value was set.
      * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
@@ -802,9 +805,10 @@
         boolean preload,
         AffinityTopologyVersion topVer,
         GridDrType drType,
-        boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        boolean fromStore,
+        boolean primary) throws IgniteCheckedException, GridCacheEntryRemovedException {
         return initialValue(val, ver, null, null, TxState.NA, TxState.NA,
-            ttl, expireTime, preload, topVer, drType, fromStore, null);
+            ttl, expireTime, preload, topVer, drType, fromStore, primary, null);
     }
 
     /**
@@ -822,6 +826,7 @@
      * @param topVer Topology version.
      * @param drType DR type.
      * @param fromStore {@code True} if value was loaded from store.
+     * @param primary {@code True} if current node is primary for partition.
      * @param row Pre-created data row, associated with this cache entry.
      * @return {@code True} if initial value was set.
      * @throws IgniteCheckedException In case of error.
@@ -839,6 +844,7 @@
         AffinityTopologyVersion topVer,
         GridDrType drType,
         boolean fromStore,
+        boolean primary,
         @Nullable CacheDataRow row) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 68c037a..54f0d88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -2152,7 +2152,7 @@
 
                 update(updated, expireTime, ttl, ver, true);
 
-                logUpdate(op, updated, ver, expireTime, 0);
+                logUpdate(op, updated, ver, expireTime, 0, true);
 
                 if (evt) {
                     CacheObject evtOld = null;
@@ -2184,7 +2184,7 @@
 
                 update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true);
 
-                logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0);
+                logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0, true);
 
                 if (evt) {
                     CacheObject evtOld = null;
@@ -3346,8 +3346,11 @@
         AffinityTopologyVersion topVer,
         GridDrType drType,
         boolean fromStore,
+        boolean primary,
         CacheDataRow row
     ) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        assert !primary || !(preload || fromStore);
+
         ensureFreeSpace();
 
         boolean deferred = false;
@@ -3499,7 +3502,8 @@
                             ver,
                             expireTime,
                             partition(),
-                            updateCntr
+                            updateCntr,
+                            DataEntry.flags(primary, preload, fromStore)
                         )));
                     }
                 }
@@ -4327,9 +4331,16 @@
      * @param writeVer Write version.
      * @param expireTime Expire time.
      * @param updCntr Update counter.
+     * @param primary {@code True} if node is primary for entry in the moment of logging.
      */
-    protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion writeVer, long expireTime, long updCntr)
-        throws IgniteCheckedException {
+    protected void logUpdate(
+        GridCacheOperation op,
+        CacheObject val,
+        GridCacheVersion writeVer,
+        long expireTime,
+        long updCntr,
+        boolean primary
+    ) throws IgniteCheckedException {
         // We log individual updates only in ATOMIC cache.
         assert cctx.atomic();
 
@@ -4344,7 +4355,8 @@
                     writeVer,
                     expireTime,
                     partition(),
-                    updCntr)));
+                    updCntr,
+                    DataEntry.flags(primary))));
         }
         catch (StorageException e) {
             throw new IgniteCheckedException("Failed to log ATOMIC cache update [key=" + key + ", op=" + op +
@@ -4359,8 +4371,12 @@
      * @param updCntr Update counter.
      * @throws IgniteCheckedException In case of log failure.
      */
-    protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr)
-        throws IgniteCheckedException {
+    protected WALPointer logTxUpdate(
+        IgniteInternalTx tx,
+        CacheObject val,
+        long expireTime,
+        long updCntr
+    ) throws IgniteCheckedException {
         assert cctx.transactional() && !cctx.transactionalSnapshot();
 
         if (tx.local()) { // For remote tx we log all updates in batch: GridDistributedTxRemoteAdapter.commitIfLocked()
@@ -4379,7 +4395,8 @@
                 tx.writeVersion(),
                 expireTime,
                 key.partition(),
-                updCntr)));
+                updCntr,
+                DataEntry.flags(CU.txOnPrimary(tx)))));
         }
         else
             return null;
@@ -6489,7 +6506,7 @@
 
             long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr);
 
-            entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0);
+            entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0, primary);
 
             if (!entry.isNear()) {
                 newRow = entry.localPartition().dataStore().createRow(
@@ -6576,7 +6593,7 @@
 
             long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr);
 
-            entry.logUpdate(op, null, newVer, 0, updateCntr0);
+            entry.logUpdate(op, null, newVer, 0, updateCntr0, primary);
 
             if (oldVal != null) {
                 assert !entry.deletedUnlocked();
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 76f8fcb..8e78b83 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
@@ -846,6 +846,17 @@
     }
 
     /**
+     * @param tx Transaction.
+     * @return {@code True} if transaction is on primary node.
+     */
+    public static boolean txOnPrimary(IgniteInternalTx tx) {
+        if (tx.near() && tx.local() && ((GridNearTxLocal)tx).colocatedLocallyMapped())
+            return true;
+
+        return tx.dht() && tx.local();
+    }
+
+    /**
      * Alias for {@link #txString(IgniteInternalTx)}.
      */
     public static String txDump(@Nullable IgniteInternalTx tx) {
@@ -1831,7 +1842,8 @@
                             true,
                             topVer,
                             GridDrType.DR_BACKUP,
-                            true);
+                            true,
+                            false);
 
                         break;
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 1f1e700..4afb684 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -631,7 +631,8 @@
                                                     writeVersion(),
                                                     0,
                                                     txEntry.key().partition(),
-                                                    txEntry.updateCounter()
+                                                    txEntry.updateCounter(),
+                                                    DataEntry.flags(CU.txOnPrimary(this))
                                                 ),
                                                 txEntry
                                             )
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 4bfb4fa..28f61ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -586,6 +586,7 @@
                         false,
                         topVer,
                         replicate ? DR_LOAD : DR_NONE,
+                        true,
                         false);
                 }
                 catch (IgniteCheckedException e) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index c9b779c..d4d71e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -1151,7 +1151,8 @@
                                     false,
                                     topVer,
                                     GridDrType.DR_LOAD,
-                                    true);
+                                    true,
+                                    false);
                             }
                             catch (GridCacheEntryRemovedException e) {
                                 assert false : "Should not get removed exception while holding lock on entry " +
@@ -1405,6 +1406,7 @@
                                     true,
                                     topVer,
                                     replicate ? DR_PRELOAD : DR_NONE,
+                                    false,
                                     false)) {
                                     if (rec && !entry.isInternal())
                                         cctx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(), null,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index ccadd5e..7c010ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -2008,6 +2008,7 @@
                                 true,
                                 topVer,
                                 drType,
+                                false,
                                 false)) {
                                 if (rec && !entry.isInternal())
                                     cacheCtx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(), null,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
index 080241b..7c66afb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
@@ -78,14 +78,14 @@
         CacheObject val,
         GridCacheVersion writeVer,
         long expireTime,
-        long updCntr
+        long updCntr,
+        boolean primary
     ) throws IgniteCheckedException {
         // No-op for detached entries, index is updated on primary or backup nodes.
     }
 
     /** {@inheritDoc} */
-    @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr)
-        throws IgniteCheckedException {
+    @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) {
         return null;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 7d16884..c0cecdc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -547,6 +547,7 @@
                             true,
                             topVer,
                             replicate ? DR_PRELOAD : DR_NONE,
+                            false,
                             false
                         )) {
                             if (rec && !entry.isInternal())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index aaad997..f4535ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -937,6 +937,7 @@
                 topVer,
                 cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE,
                 false,
+                false,
                 row
             )) {
                 cached.touch(); // Start tracking.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index d8b1615..a948dab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -471,14 +471,13 @@
     }
 
     /** {@inheritDoc} */
-    @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion ver, long expireTime, long updCntr)
-        throws IgniteCheckedException {
+    @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion ver, long expireTime,
+        long updCntr, boolean primary) {
         // No-op: queries are disabled for near cache.
     }
 
     /** {@inheritDoc} */
-    @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr)
-        throws IgniteCheckedException {
+    @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) {
         return null;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java
new file mode 100644
index 0000000..2e8a88b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java
@@ -0,0 +1,202 @@
+/*
+ * 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.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+import java.nio.file.Paths;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract file lock holder.
+ * Implementations should provide {@link #lockInfo()} that will appear in error message for concurrent processes
+ * that will try to lock the same file and {@link #warningMessage(String)} to print on each lock try.
+ *
+ * @see GridCacheDatabaseSharedManager.NodeFileLockHolder
+ */
+public abstract class FileLockHolder implements AutoCloseable {
+    /** Lock file name. */
+    private static final String lockFileName = "lock";
+
+    /** File. */
+    private final File file;
+
+    /** Channel. */
+    private final RandomAccessFile lockFile;
+
+    /** Lock. */
+    private volatile FileLock lock;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /**
+     * @param rootDir Root directory for lock file.
+     * @param log Log.
+     */
+    protected FileLockHolder(String rootDir, IgniteLogger log) {
+        try {
+            file = Paths.get(rootDir, lockFileName).toFile();
+
+            lockFile = new RandomAccessFile(file, "rw");
+
+            this.log = log;
+        }
+        catch (IOException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * This info will appear in error message of concurrent processes that will try to lock on the same file.
+     *
+     * @return Lock info to store in the file.
+     */
+    public abstract String lockInfo();
+
+    /**
+     * @param lockInfo Existing lock info.
+     * @return Warning message.
+     */
+    protected abstract String warningMessage(String lockInfo);
+
+    /**
+     * @param lockWaitTimeMillis During which time thread will try capture file lock.
+     * @throws IgniteCheckedException If failed to capture file lock.
+     */
+    public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException {
+        assert lockFile != null;
+
+        FileChannel ch = lockFile.getChannel();
+
+        String failMsg;
+
+        try {
+            String content = null;
+
+            // Try to get lock, if not available wait 1 sec and re-try.
+            for (int i = 0; i < lockWaitTimeMillis; i += 1000) {
+                try {
+                    lock = ch.tryLock(0, 1, false);
+
+                    if (lock != null && lock.isValid()) {
+                        writeContent(lockInfo());
+
+                        return;
+                    }
+                }
+                catch (OverlappingFileLockException ignore) {
+                    if (content == null)
+                        content = readContent();
+
+                    log.warning(warningMessage(content));
+                }
+
+                U.sleep(1000);
+            }
+
+            if (content == null)
+                content = readContent();
+
+            failMsg = "Failed to acquire file lock [holder=" + content + ", time=" + (lockWaitTimeMillis / 1000) +
+                " sec, path=" + file.getAbsolutePath() + ']';
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        if (failMsg != null)
+            throw new IgniteCheckedException(failMsg);
+    }
+
+    /**
+     * Write node id (who captured lock) into lock file.
+     *
+     * @param content Node id.
+     * @throws IOException if some fail while write node it.
+     */
+    private void writeContent(String content) throws IOException {
+        FileChannel ch = lockFile.getChannel();
+
+        byte[] bytes = content.getBytes();
+
+        ByteBuffer buf = ByteBuffer.allocate(bytes.length);
+        buf.put(bytes);
+
+        buf.flip();
+
+        ch.write(buf, 1);
+
+        ch.force(false);
+    }
+
+    /**
+     *
+     */
+    private String readContent() throws IOException {
+        FileChannel ch = lockFile.getChannel();
+
+        ByteBuffer buf = ByteBuffer.allocate((int)(ch.size() - 1));
+
+        ch.read(buf, 1);
+
+        String content = new String(buf.array());
+
+        buf.clear();
+
+        return content;
+    }
+
+    /**
+     * Locked or not.
+     */
+    public boolean isLocked() {
+        return lock != null && lock.isValid();
+    }
+
+    /**
+     * Releases file lock
+     */
+    public void release() {
+        U.releaseQuiet(lock);
+    }
+
+    /**
+     * Closes file channel
+     */
+    @Override public void close() {
+        release();
+
+        U.closeQuiet(lockFile);
+    }
+
+    /**
+     * @return Absolute path to lock file.
+     */
+    public String lockPath() {
+        return file.getAbsolutePath();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index b7f9625..5544244 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -19,15 +19,10 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -306,7 +301,7 @@
      * Lock holder for compatible folders mode. Null if lock holder was created at start node. <br>
      * In this case lock is held on PDS resover manager and it is not required to manage locking here
      */
-    @Nullable private FileLockHolder fileLockHolder;
+    @Nullable private NodeFileLockHolder fileLockHolder;
 
     /** Lock wait time. */
     private final long lockWaitTime;
@@ -560,7 +555,7 @@
                 () -> cpFreqDeviation.getOrDefault(DEFAULT_CHECKPOINT_DEVIATION)
             );
 
-            final FileLockHolder preLocked = kernalCtx.pdsFolderResolver()
+            final NodeFileLockHolder preLocked = kernalCtx.pdsFolderResolver()
                 .resolveFolders()
                 .getLockedFileLockHolder();
 
@@ -744,12 +739,12 @@
     /**
      * @param preLocked Pre-locked file lock holder.
      */
-    private void acquireFileLock(FileLockHolder preLocked) throws IgniteCheckedException {
+    private void acquireFileLock(NodeFileLockHolder preLocked) throws IgniteCheckedException {
         if (cctx.kernalContext().clientNode())
             return;
 
         fileLockHolder = preLocked == null ?
-            new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log) : preLocked;
+            new NodeFileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log) : preLocked;
 
         if (!fileLockHolder.isLocked()) {
             if (log.isDebugEnabled())
@@ -2547,6 +2542,7 @@
                 switch (rec.type()) {
                     case MVCC_DATA_RECORD:
                     case DATA_RECORD:
+                    case DATA_RECORD_V2:
                         checkpointReadLock();
 
                         try {
@@ -2697,8 +2693,10 @@
 
                     case MVCC_DATA_RECORD:
                     case DATA_RECORD:
+                    case DATA_RECORD_V2:
                     case ENCRYPTED_DATA_RECORD:
                     case ENCRYPTED_DATA_RECORD_V2:
+                    case ENCRYPTED_DATA_RECORD_V3:
                         DataRecord dataRec = (DataRecord)rec;
 
                         for (DataEntry dataEntry : dataRec.writeEntries()) {
@@ -3008,53 +3006,25 @@
     }
 
     /**
-     *
+     * Node file lock holder.
      */
-    public static class FileLockHolder implements AutoCloseable {
-        /** Lock file name. */
-        private static final String lockFileName = "lock";
-
-        /** File. */
-        private final File file;
-
-        /** Channel. */
-        private final RandomAccessFile lockFile;
-
-        /** Lock. */
-        private volatile FileLock lock;
-
+    public static class NodeFileLockHolder extends FileLockHolder {
         /** Kernal context to generate Id of locked node in file. */
         @NotNull private final GridKernalContext ctx;
 
-        /** Logger. */
-        private final IgniteLogger log;
-
         /**
-         * @param path Path.
+         * @param rootDir Root directory for lock file.
+         * @param ctx Kernal context.
+         * @param log Log.
          */
-        public FileLockHolder(String path, @NotNull GridKernalContext ctx, IgniteLogger log) {
-            try {
-                file = Paths.get(path, lockFileName).toFile();
+        public NodeFileLockHolder(String rootDir, @NotNull GridKernalContext ctx, IgniteLogger log) {
+            super(rootDir, log);
 
-                lockFile = new RandomAccessFile(file, "rw");
-
-                this.ctx = ctx;
-                this.log = log;
-            }
-            catch (IOException e) {
-                throw new IgniteException(e);
-            }
+            this.ctx = ctx;
         }
 
-        /**
-         * @param lockWaitTimeMillis During which time thread will try capture file lock.
-         * @throws IgniteCheckedException If failed to capture file lock.
-         */
-        public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException {
-            assert lockFile != null;
-
-            FileChannel ch = lockFile.getChannel();
-
+        /** {@inheritDoc} */
+        @Override public String lockInfo() {
             SB sb = new SB();
 
             //write node id
@@ -3085,108 +3055,14 @@
 
             sb.a("]");
 
-            String failMsg;
-
-            try {
-                String content = null;
-
-                // Try to get lock, if not available wait 1 sec and re-try.
-                for (int i = 0; i < lockWaitTimeMillis; i += 1000) {
-                    try {
-                        lock = ch.tryLock(0, 1, false);
-
-                        if (lock != null && lock.isValid()) {
-                            writeContent(sb.toString());
-
-                            return;
-                        }
-                    }
-                    catch (OverlappingFileLockException ignore) {
-                        if (content == null)
-                            content = readContent();
-
-                        log.warning("Failed to acquire file lock. Will try again in 1s " +
-                            "[nodeId=" + ctx.localNodeId() + ", holder=" + content +
-                            ", path=" + file.getAbsolutePath() + ']');
-                    }
-
-                    U.sleep(1000);
-                }
-
-                if (content == null)
-                    content = readContent();
-
-                failMsg = "Failed to acquire file lock [holder=" + content + ", time=" + (lockWaitTimeMillis / 1000) +
-                    " sec, path=" + file.getAbsolutePath() + ']';
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException(e);
-            }
-
-            if (failMsg != null)
-                throw new IgniteCheckedException(failMsg);
+            return sb.toString();
         }
 
-        /**
-         * Write node id (who captured lock) into lock file.
-         *
-         * @param content Node id.
-         * @throws IOException if some fail while write node it.
-         */
-        private void writeContent(String content) throws IOException {
-            FileChannel ch = lockFile.getChannel();
-
-            byte[] bytes = content.getBytes();
-
-            ByteBuffer buf = ByteBuffer.allocate(bytes.length);
-            buf.put(bytes);
-
-            buf.flip();
-
-            ch.write(buf, 1);
-
-            ch.force(false);
-        }
-
-        /**
-         *
-         */
-        private String readContent() throws IOException {
-            FileChannel ch = lockFile.getChannel();
-
-            ByteBuffer buf = ByteBuffer.allocate((int)(ch.size() - 1));
-
-            ch.read(buf, 1);
-
-            String content = new String(buf.array());
-
-            buf.clear();
-
-            return content;
-        }
-
-        /** Locked or not. */
-        public boolean isLocked() {
-            return lock != null && lock.isValid();
-        }
-
-        /** Releases file lock */
-        public void release() {
-            U.releaseQuiet(lock);
-        }
-
-        /** Closes file channel */
-        @Override public void close() {
-            release();
-
-            U.closeQuiet(lockFile);
-        }
-
-        /**
-         * @return Absolute path to lock file.
-         */
-        private String lockPath() {
-            return file.getAbsolutePath();
+        /** {@inheritDoc} */
+        @Override protected String warningMessage(String lockInfo) {
+            return "Failed to acquire file lock. Will try again in 1s " +
+                "[nodeId=" + ctx.localNodeId() + ", holder=" + lockInfo +
+                ", path=" + lockPath() + ']';
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
index 951a2e1..f6a9752 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
@@ -18,89 +18,25 @@
 package org.apache.ignite.internal.processors.cache.persistence.filename;
 
 import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.Serializable;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.UUID;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
-import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 
 /**
  * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled
  */
 public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver {
-    /** Database subfolders constant prefix. */
-    private static final String DB_FOLDER_PREFIX = "node";
-
-    /** Node index and uid separator in subfolders name. */
-    private static final String NODEIDX_UID_SEPARATOR = "-";
-
-    /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */
-    private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR;
-
-    /** Uuid as string pattern. */
-    private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
-
-    /**
-     * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID)
-     * pattern.
-     */
-    private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN;
-
-    /** Database subfolders for new style filter. */
-    public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() {
-        @Override public boolean accept(File pathname) {
-            return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN);
-        }
-    };
-
-    /** Database subfolders for old style filter. */
-    private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() {
-        @Override public boolean accept(File pathname) {
-            String path = pathname.toString();
-            return pathname.isDirectory()
-                && !"wal".equals(pathname.getName())
-                && !path.contains(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH)
-                && !path.contains(DataStorageConfiguration.DFLT_MARSHALLER_PATH)
-                && !pathname.getName().matches(SUBDIR_PATTERN);
-        }
-    };
-
-    /** Database default folder. */
-    public static final String DB_DEFAULT_FOLDER = "db";
-
-    /** Config. */
-    private IgniteConfiguration cfg;
-
     /** Logger. */
-    private IgniteLogger log;
+    private final IgniteLogger log;
 
     /** Context. */
-    private GridKernalContext ctx;
+    private final GridKernalContext ctx;
 
     /** Cached folder settings. */
-    private PdsFolderSettings settings;
+    private PdsFolderSettings<NodeFileLockHolder> settings;
 
     /**
      * Creates folders resolver
@@ -110,34 +46,21 @@
     public PdsConsistentIdProcessor(final GridKernalContext ctx) {
         super(ctx);
 
-        this.cfg = ctx.config();
         this.log = ctx.log(PdsFoldersResolver.class);
         this.ctx = ctx;
     }
 
-    /**
-     * Prepares compatible PDS folder settings. No locking is performed, consistent ID is not overridden.
-     *
-     * @param pstStoreBasePath DB storage base path or null if persistence is not enabled.
-     * @param consistentId compatibility consistent ID
-     * @return PDS folder settings compatible with previous versions.
-     */
-    private PdsFolderSettings compatibleResolve(
-        @Nullable final File pstStoreBasePath,
-        @NotNull final Serializable consistentId) {
-
-        if (cfg.getConsistentId() != null) {
-            // compatible mode from configuration is used fot this case, no locking, no consitent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
-        }
-
-        return new PdsFolderSettings(pstStoreBasePath, consistentId);
-    }
-
     /** {@inheritDoc} */
-    @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException {
+    @Override public PdsFolderSettings<NodeFileLockHolder> resolveFolders() throws IgniteCheckedException {
         if (settings == null) {
-            settings = prepareNewSettings();
+            //here deprecated method is used to get compatible version of consistentId
+            PdsFolderResolver<NodeFileLockHolder> resolver =
+                new PdsFolderResolver<>(ctx.config(), log, ctx.discovery().consistentId(), this::tryLock);
+
+            settings = resolver.resolve();
+
+            if (settings == null)
+                settings = resolver.generateNew();
 
             if (!settings.isCompatible()) {
                 if (log.isInfoEnabled())
@@ -151,262 +74,6 @@
     }
 
     /**
-     * Creates new settings when we don't have cached one.
-     *
-     * @return new settings with prelocked directory (if appropriate).
-     * @throws IgniteCheckedException if IO failed.
-     */
-    private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException {
-        final File pstStoreBasePath = resolvePersistentStoreBasePath();
-        //here deprecated method is used to get compatible version of consistentId
-        final Serializable consistentId = ctx.discovery().consistentId();
-
-        if (!CU.isPersistenceEnabled(cfg))
-            return compatibleResolve(pstStoreBasePath, consistentId);
-
-        if (ctx.clientNode())
-            return new PdsFolderSettings(pstStoreBasePath, UUID.randomUUID());
-
-        if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
-            return compatibleResolve(pstStoreBasePath, consistentId);
-
-        // compatible mode from configuration is used fot this case
-        if (cfg.getConsistentId() != null) {
-            // compatible mode from configuration is used fot this case, no locking, no consistent id change
-            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
-        }
-        // The node scans the work directory and checks if there is a folder matching the consistent ID.
-        // If such a folder exists, we start up with this ID (compatibility mode)
-        final String subFolder = U.maskForFileName(consistentId.toString());
-
-        final GridCacheDatabaseSharedManager.FileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder));
-
-        if (oldStyleFolderLockHolder != null)
-            return new PdsFolderSettings(pstStoreBasePath,
-                subFolder,
-                consistentId,
-                oldStyleFolderLockHolder,
-                true);
-
-        final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER);
-
-        if (oldStyleFolders != null && oldStyleFolders.length != 0) {
-            for (File folder : oldStyleFolders) {
-                final String path = getPathDisplayableInfo(folder);
-
-                U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]");
-            }
-        }
-
-        for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) {
-            final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(next.subFolderFile());
-
-            if (fileLockHolder != null) {
-                if (log.isInfoEnabled())
-                    log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]");
-
-                return new PdsFolderSettings(pstStoreBasePath,
-                    next.subFolderFile().getName(),
-                    next.uuid(),
-                    fileLockHolder,
-                    false);
-            }
-        }
-
-        // was not able to find free slot, allocating new
-        try (final GridCacheDatabaseSharedManager.FileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) {
-            final List<FolderCandidate> sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath);
-            final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1);
-
-            return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx);
-        }
-    }
-
-    /**
-     * Calculate overall folder size.
-     *
-     * @param dir directory to scan.
-     * @return total size in bytes.
-     */
-    private static FolderParams folderSize(File dir) {
-        final FolderParams params = new FolderParams();
-
-        visitFolder(dir, params);
-
-        return params;
-    }
-
-    /**
-     * Scans provided directory and its sub dirs, collects found metrics.
-     *
-     * @param dir directory to start scan from.
-     * @param params input/output.
-     */
-    private static void visitFolder(final File dir, final FolderParams params) {
-        for (File file : dir.listFiles()) {
-            if (file.isDirectory())
-                visitFolder(file, params);
-            else {
-                params.size += file.length();
-                params.lastModified = Math.max(params.lastModified, dir.lastModified());
-            }
-        }
-    }
-
-    /**
-     * @param folder folder to scan.
-     * @return folder displayable information.
-     */
-    @NotNull private String getPathDisplayableInfo(final File folder) {
-        final SB res = new SB();
-
-        res.a(getCanonicalPath(folder));
-        res.a(", ");
-        final FolderParams params = folderSize(folder);
-
-        res.a(params.size);
-        res.a(" bytes, modified ");
-        final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a");
-
-        res.a(simpleDateFormat.format(params.lastModified));
-        res.a(" ");
-
-        return res.toString();
-    }
-
-    /**
-     * Returns the canonical pathname string of this abstract pathname.
-     *
-     * @param file path to convert.
-     * @return canonical pathname or at leas absolute if convert to canonical failed.
-     */
-    @NotNull private String getCanonicalPath(final File file) {
-        try {
-            return file.getCanonicalPath();
-        }
-        catch (IOException ignored) {
-            return file.getAbsolutePath();
-        }
-    }
-
-    /**
-     * Pad start of string with provided character.
-     *
-     * @param str sting to pad.
-     * @param minLength expected length.
-     * @param padChar padding character.
-     * @return padded string.
-     */
-    private static String padStart(String str, int minLength, char padChar) {
-        A.notNull(str, "String should not be empty");
-        if (str.length() >= minLength)
-            return str;
-
-        final SB sb = new SB(minLength);
-
-        for (int i = str.length(); i < minLength; ++i)
-            sb.a(padChar);
-
-        sb.a(str);
-
-        return sb.toString();
-
-    }
-
-    /**
-     * Creates new DB storage folder.
-     *
-     * @param pstStoreBasePath DB root path.
-     * @param nodeIdx next node index to use in folder name.
-     * @return new settings to be used in this node.
-     * @throws IgniteCheckedException if failed.
-     */
-    @NotNull private PdsFolderSettings generateAndLockNewDbStorage(final File pstStoreBasePath,
-        final int nodeIdx) throws IgniteCheckedException {
-
-        final UUID uuid = UUID.randomUUID();
-        final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid);
-        final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here
-        final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(newRandomFolder);
-
-        if (fileLockHolder != null) {
-            if (log.isInfoEnabled())
-                log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]");
-
-            return new PdsFolderSettings(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
-        }
-        throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]");
-    }
-
-    /**
-     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
-     *
-     * @param nodeIdx node index.
-     * @param uuid consistent ID.
-     * @return folder file name
-     */
-    @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) {
-        final String uuidAsStr = uuid.toString();
-
-        assert uuidAsStr.matches(UUID_STR_PATTERN);
-
-        final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0');
-
-        return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr;
-    }
-
-    /**
-     * Acquires lock to root storage directory, used to lock root directory in case creating new files is required.
-     *
-     * @param pstStoreBasePath rood DB dir to lock
-     * @return locked directory, should be released and closed later
-     * @throws IgniteCheckedException if failed
-     */
-    @NotNull private GridCacheDatabaseSharedManager.FileLockHolder lockRootDirectory(File pstStoreBasePath)
-        throws IgniteCheckedException {
-
-        GridCacheDatabaseSharedManager.FileLockHolder rootDirLock;
-        int retry = 0;
-
-        while ((rootDirLock = tryLock(pstStoreBasePath)) == null) {
-            if (retry > 600)
-                throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" +
-                    ". Lock is being held to root directory");
-            retry++;
-        }
-
-        return rootDirLock;
-    }
-
-    /**
-     * @param pstStoreBasePath root storage folder to scan.
-     * @return empty list if there is no files in folder to test. Non null value is returned for folder having
-     * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first.
-     */
-    @Nullable private List<FolderCandidate> getNodeIndexSortedCandidates(File pstStoreBasePath) {
-        final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER);
-
-        if (files == null)
-            return Collections.emptyList();
-
-        final List<FolderCandidate> res = new ArrayList<>();
-
-        for (File file : files) {
-            final FolderCandidate candidate = parseFileName(file);
-
-            if (candidate != null)
-                res.add(candidate);
-        }
-        Collections.sort(res, new Comparator<FolderCandidate>() {
-            @Override public int compare(FolderCandidate c1, FolderCandidate c2) {
-                return Integer.compare(c1.nodeIndex(), c2.nodeIndex());
-            }
-        });
-
-        return res;
-    }
-
-    /**
      * Tries to lock subfolder within storage root folder.
      *
      * @param dbStoreDirWithSubdirectory DB store directory, is to be absolute and should include consistent ID based
@@ -414,13 +81,12 @@
      * @return non null holder if lock was successful, null in case lock failed. If directory does not exist method will
      * always fail to lock.
      */
-    private GridCacheDatabaseSharedManager.FileLockHolder tryLock(File dbStoreDirWithSubdirectory) {
+    private NodeFileLockHolder tryLock(File dbStoreDirWithSubdirectory) {
         if (!dbStoreDirWithSubdirectory.exists())
             return null;
 
         final String path = dbStoreDirWithSubdirectory.getAbsolutePath();
-        final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder
-            = new GridCacheDatabaseSharedManager.FileLockHolder(path, ctx, log);
+        final NodeFileLockHolder fileLockHolder = new NodeFileLockHolder(path, ctx, log);
 
         try {
             fileLockHolder.tryLock(1000);
@@ -437,70 +103,10 @@
         }
     }
 
-    /**
-     * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
-     * store configuration. Null if persistence is not enabled. Returned folder is created automatically.
-     * @throws IgniteCheckedException if I/O failed.
-     */
-    @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException {
-        final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration();
-
-        if (dsCfg == null)
-            return null;
-
-        final String pstPath = dsCfg.getStoragePath();
-
-        return U.resolveWorkDirectory(
-            cfg.getWorkDirectory(),
-            pstPath != null ? pstPath : DB_DEFAULT_FOLDER,
-            false
-        );
-
-    }
-
-    /**
-     * @param subFolderFile new style folder name to parse
-     * @return Pair of UUID and node index
-     */
-    private FolderCandidate parseFileName(@NotNull final File subFolderFile) {
-        return parseSubFolderName(subFolderFile, log);
-    }
-
-    /**
-     * @param subFolderFile new style file to parse.
-     * @param log Logger.
-     * @return Pair of UUID and node index.
-     */
-    @Nullable public static FolderCandidate parseSubFolderName(
-        @NotNull final File subFolderFile,
-        @NotNull final IgniteLogger log) {
-
-        final String fileName = subFolderFile.getName();
-        final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName);
-        if (!matcher.find())
-            return null;
-
-        int uidStart = matcher.end();
-
-        try {
-            final String uid = fileName.substring(uidStart);
-            final UUID uuid = UUID.fromString(uid);
-            final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length());
-            final int idx = Integer.parseInt(substring);
-
-            return new FolderCandidate(subFolderFile, idx, uuid);
-        }
-        catch (Exception e) {
-            U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e);
-
-            return null;
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (settings != null) {
-            final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = settings.getLockedFileLockHolder();
+            final NodeFileLockHolder fileLockHolder = settings.getLockedFileLockHolder();
 
             if (fileLockHolder != null)
                 fileLockHolder.close();
@@ -508,61 +114,6 @@
 
         super.stop(cancel);
     }
-
-    /** Path metrics */
-    private static class FolderParams {
-        /** Overall size in bytes. */
-        private long size;
-
-        /** Last modified. */
-        private long lastModified;
-    }
-
-    /**
-     * Represents parsed new style file and encoded parameters in this file name
-     */
-    public static class FolderCandidate {
-        /** Absolute file path pointing to DB subfolder within DB storage root folder. */
-        private final File subFolderFile;
-
-        /** Node index (local, usually 0 if multiple nodes are not started at local PC). */
-        private final int nodeIdx;
-
-        /** Uuid contained in file name, is to be set as consistent ID. */
-        private final UUID uuid;
-
-        /**
-         * @param subFolderFile Absolute file path pointing to DB subfolder.
-         * @param nodeIdx Node index.
-         * @param uuid Uuid.
-         */
-        public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) {
-            this.subFolderFile = subFolderFile;
-            this.nodeIdx = nodeIdx;
-            this.uuid = uuid;
-        }
-
-        /**
-         * @return Node index (local, usually 0 if multiple nodes are not started at local PC).
-         */
-        public int nodeIndex() {
-            return nodeIdx;
-        }
-
-        /**
-         * @return Uuid contained in file name, is to be set as consistent ID.
-         */
-        public Serializable uuid() {
-            return uuid;
-        }
-
-        /**
-         * @return Absolute file path pointing to DB subfolder within DB storage root folder.
-         */
-        public File subFolderFile() {
-            return subFolderFile;
-        }
-    }
 }
 
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java
new file mode 100644
index 0000000..ce0a8ae
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java
@@ -0,0 +1,528 @@
+/*
+ * 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.persistence.filename;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.Serializable;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static java.lang.Boolean.TRUE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+
+/**
+ * This class contains logic to resolve and possibly lock PDS folder
+ * based on provided {@link IgniteConfiguration} and {@link #consistentId}.
+ *
+ * @param <L> Type of the lock holder.
+ */
+public class PdsFolderResolver<L extends FileLockHolder> {
+    /** Database subfolders constant prefix. */
+    private static final String DB_FOLDER_PREFIX = "node";
+
+    /** Node index and uid separator in subfolders name. */
+    private static final String NODEIDX_UID_SEPARATOR = "-";
+
+    /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */
+    private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR;
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /**
+     * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID)
+     * pattern.
+     */
+    private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN;
+
+    /** Database subfolders for new style filter. */
+    public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() {
+        @Override public boolean accept(File pathname) {
+            return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN);
+        }
+    };
+
+    /** Database subfolders for old style filter. */
+    private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() {
+        @Override public boolean accept(File pathname) {
+            String path = pathname.toString();
+            return pathname.isDirectory()
+                && !"wal".equals(pathname.getName())
+                && !path.contains(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH)
+                && !path.contains(DataStorageConfiguration.DFLT_MARSHALLER_PATH)
+                && !pathname.getName().matches(SUBDIR_PATTERN);
+        }
+    };
+
+    /** Database default folder. */
+    public static final String DB_DEFAULT_FOLDER = "db";
+
+    /** */
+    private final IgniteConfiguration cfg;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final @Nullable Serializable consistentId;
+
+    /** */
+    private final Function<File, L> tryLock;
+
+    /**
+     * @param cfg Ignite configuration.
+     * @param log Logger.
+     * @param consistentId Constent id.
+     * @param tryLock Lock function.
+     */
+    public PdsFolderResolver(
+        IgniteConfiguration cfg,
+        IgniteLogger log,
+        @Nullable Serializable consistentId,
+        Function<File, L> tryLock
+    ) {
+        this.cfg = cfg;
+        this.log = log;
+        this.consistentId = consistentId;
+        this.tryLock = tryLock;
+    }
+
+    /**
+     * Prepares compatible PDS folder settings. No locking is performed, consistent ID is not overridden.
+     *
+     * @param pstStoreBasePath DB storage base path or null if persistence is not enabled.
+     * @param consistentId compatibility consistent ID
+     * @return PDS folder settings compatible with previous versions.
+     */
+    private PdsFolderSettings<L> compatibleResolve(
+        @Nullable final File pstStoreBasePath,
+        @NotNull final Serializable consistentId) {
+
+        if (cfg.getConsistentId() != null) {
+            // compatible mode from configuration is used fot this case, no locking, no consitent id change
+            return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId());
+        }
+
+        return new PdsFolderSettings<>(pstStoreBasePath, consistentId);
+    }
+
+    /**
+     * Resolves {@link PdsFolderSettings} according to specified {@link IgniteConfiguration}, {@link #consistentId}.
+     *
+     * @return new settings with prelocked directory (if appropriate) or null.
+     * @throws IgniteCheckedException if IO failed.
+     */
+    public PdsFolderSettings<L> resolve() throws IgniteCheckedException {
+        final File pstStoreBasePath = resolvePersistentStoreBasePath();
+
+        if (!CU.isPersistenceEnabled(cfg))
+            return compatibleResolve(pstStoreBasePath, consistentId);
+
+        if (cfg.isClientMode() == TRUE || cfg.isDaemon())
+            return new PdsFolderSettings<>(pstStoreBasePath, UUID.randomUUID());
+
+        if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
+            return compatibleResolve(pstStoreBasePath, consistentId);
+
+        // compatible mode from configuration is used fot this case
+        if (cfg.getConsistentId() != null) {
+            // compatible mode from configuration is used fot this case, no locking, no consistent id change
+            return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId());
+        }
+
+        if (consistentId != null) {
+            // The node scans the work directory and checks if there is a folder matching the consistent ID.
+            // If such a folder exists, we start up with this ID (compatibility mode)
+            final String subFolder = U.maskForFileName(consistentId.toString());
+
+            final L oldStyleFolderLockHolder = tryLock.apply(new File(pstStoreBasePath, subFolder));
+
+            if (oldStyleFolderLockHolder != null) {
+                return new PdsFolderSettings<>(pstStoreBasePath,
+                    subFolder,
+                    consistentId,
+                    oldStyleFolderLockHolder,
+                    true);
+            }
+        }
+
+        final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER);
+
+        if (oldStyleFolders != null && oldStyleFolders.length != 0) {
+            for (File folder : oldStyleFolders) {
+                final String path = getPathDisplayableInfo(folder);
+
+                U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]");
+            }
+        }
+
+        for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) {
+            final L fileLockHolder = tryLock.apply(next.subFolderFile());
+
+            if (fileLockHolder != null) {
+                if (log.isInfoEnabled())
+                    log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]");
+
+                return new PdsFolderSettings<>(pstStoreBasePath,
+                    next.subFolderFile().getName(),
+                    next.uuid(),
+                    fileLockHolder,
+                    false);
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * @return New PDS folder.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public PdsFolderSettings<L> generateNew() throws IgniteCheckedException {
+        final File pstStoreBasePath = resolvePersistentStoreBasePath();
+
+        // was not able to find free slot, allocating new
+        try (final L rootDirLock = lockRootDirectory(pstStoreBasePath)) {
+            final List<FolderCandidate> sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath);
+            final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1);
+
+            return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx);
+        }
+    }
+
+    /**
+     * Calculate overall folder size.
+     *
+     * @param dir directory to scan.
+     * @return total size in bytes.
+     */
+    private static FolderParams folderSize(File dir) {
+        final FolderParams params = new FolderParams();
+
+        visitFolder(dir, params);
+
+        return params;
+    }
+
+    /**
+     * Scans provided directory and its sub dirs, collects found metrics.
+     *
+     * @param dir directory to start scan from.
+     * @param params input/output.
+     */
+    private static void visitFolder(final File dir, final FolderParams params) {
+        for (File file : dir.listFiles()) {
+            if (file.isDirectory())
+                visitFolder(file, params);
+            else {
+                params.size += file.length();
+                params.lastModified = Math.max(params.lastModified, dir.lastModified());
+            }
+        }
+    }
+
+    /**
+     * @param folder folder to scan.
+     * @return folder displayable information.
+     */
+    @NotNull private String getPathDisplayableInfo(final File folder) {
+        final SB res = new SB();
+
+        res.a(getCanonicalPath(folder));
+        res.a(", ");
+        final FolderParams params = folderSize(folder);
+
+        res.a(params.size);
+        res.a(" bytes, modified ");
+        final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a");
+
+        res.a(simpleDateFormat.format(params.lastModified));
+        res.a(" ");
+
+        return res.toString();
+    }
+
+    /**
+     * Returns the canonical pathname string of this abstract pathname.
+     *
+     * @param file path to convert.
+     * @return canonical pathname or at leas absolute if convert to canonical failed.
+     */
+    @NotNull private String getCanonicalPath(final File file) {
+        try {
+            return file.getCanonicalPath();
+        }
+        catch (IOException ignored) {
+            return file.getAbsolutePath();
+        }
+    }
+
+    /**
+     * Pad start of string with provided character.
+     *
+     * @param str sting to pad.
+     * @param minLength expected length.
+     * @param padChar padding character.
+     * @return padded string.
+     */
+    private static String padStart(String str, int minLength, char padChar) {
+        A.notNull(str, "String should not be empty");
+        if (str.length() >= minLength)
+            return str;
+
+        final SB sb = new SB(minLength);
+
+        for (int i = str.length(); i < minLength; ++i)
+            sb.a(padChar);
+
+        sb.a(str);
+
+        return sb.toString();
+    }
+
+    /**
+     * Creates new DB storage folder.
+     *
+     * @param pstStoreBasePath DB root path.
+     * @param nodeIdx next node index to use in folder name.
+     * @return new settings to be used in this node.
+     * @throws IgniteCheckedException if failed.
+     */
+    @NotNull private PdsFolderSettings<L> generateAndLockNewDbStorage(
+        final File pstStoreBasePath,
+        final int nodeIdx
+    ) throws IgniteCheckedException {
+        final UUID uuid = UUID.randomUUID();
+        final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid);
+        final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here
+        final L fileLockHolder = tryLock.apply(newRandomFolder);
+
+        if (fileLockHolder != null) {
+            if (log.isInfoEnabled())
+                log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]");
+
+            return new PdsFolderSettings<>(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
+        }
+
+        throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]");
+    }
+
+    /**
+     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
+     *
+     * @param nodeIdx node index.
+     * @param uuid consistent ID.
+     * @return folder file name
+     */
+    @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) {
+        final String uuidAsStr = uuid.toString();
+
+        assert uuidAsStr.matches(UUID_STR_PATTERN);
+
+        final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0');
+
+        return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr;
+    }
+
+    /**
+     * Acquires lock to root storage directory, used to lock root directory in case creating new files is required.
+     *
+     * @param pstStoreBasePath rood DB dir to lock
+     * @return locked directory, should be released and closed later
+     * @throws IgniteCheckedException if failed
+     */
+    @NotNull private L lockRootDirectory(File pstStoreBasePath)
+        throws IgniteCheckedException {
+
+        L rootDirLock;
+        int retry = 0;
+
+        while ((rootDirLock = tryLock.apply(pstStoreBasePath)) == null) {
+            if (retry > 600)
+                throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" +
+                    ". Lock is being held to root directory");
+            retry++;
+        }
+
+        return rootDirLock;
+    }
+
+    /**
+     * @param pstStoreBasePath root storage folder to scan.
+     * @return empty list if there is no files in folder to test. Non null value is returned for folder having
+     * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first.
+     */
+    @Nullable private List<FolderCandidate> getNodeIndexSortedCandidates(File pstStoreBasePath) {
+        final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER);
+
+        if (files == null)
+            return Collections.emptyList();
+
+        final List<FolderCandidate> res = new ArrayList<>();
+
+        for (File file : files) {
+            final FolderCandidate candidate = parseFileName(file);
+
+            if (candidate != null)
+                res.add(candidate);
+        }
+
+        Collections.sort(res, new Comparator<FolderCandidate>() {
+            @Override public int compare(FolderCandidate c1, FolderCandidate c2) {
+                return Integer.compare(c1.nodeIndex(), c2.nodeIndex());
+            }
+        });
+
+        return res;
+    }
+
+    /**
+     * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
+     * store configuration. Null if persistence is not enabled. Returned folder is created automatically.
+     * @throws IgniteCheckedException if I/O failed.
+     */
+    @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException {
+        final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration();
+
+        if (dsCfg == null)
+            return null;
+
+        final String pstPath = dsCfg.getStoragePath();
+
+        return U.resolveWorkDirectory(
+            cfg.getWorkDirectory(),
+            pstPath != null ? pstPath : DB_DEFAULT_FOLDER,
+            false
+        );
+    }
+
+    /**
+     * @param subFolderFile new style folder name to parse
+     * @return Pair of UUID and node index
+     */
+    private FolderCandidate parseFileName(@NotNull final File subFolderFile) {
+        return parseSubFolderName(subFolderFile, log);
+    }
+
+    /**
+     * @param subFolderFile new style file to parse.
+     * @param log Logger.
+     * @return Pair of UUID and node index.
+     */
+    @Nullable public static FolderCandidate parseSubFolderName(
+        @NotNull final File subFolderFile,
+        @NotNull final IgniteLogger log) {
+
+        final String fileName = subFolderFile.getName();
+        final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName);
+        if (!matcher.find())
+            return null;
+
+        int uidStart = matcher.end();
+
+        try {
+            final String uid = fileName.substring(uidStart);
+            final UUID uuid = UUID.fromString(uid);
+            final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length());
+            final int idx = Integer.parseInt(substring);
+
+            return new FolderCandidate(subFolderFile, idx, uuid);
+        }
+        catch (Exception e) {
+            U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e);
+
+            return null;
+        }
+    }
+
+    /** Path metrics */
+    private static class FolderParams {
+        /** Overall size in bytes. */
+        private long size;
+
+        /** Last modified. */
+        private long lastModified;
+    }
+
+    /**
+     * Represents parsed new style file and encoded parameters in this file name
+     */
+    public static class FolderCandidate {
+        /** Absolute file path pointing to DB subfolder within DB storage root folder. */
+        private final File subFolderFile;
+
+        /** Node index (local, usually 0 if multiple nodes are not started at local PC). */
+        private final int nodeIdx;
+
+        /** Uuid contained in file name, is to be set as consistent ID. */
+        private final UUID uuid;
+
+        /**
+         * @param subFolderFile Absolute file path pointing to DB subfolder.
+         * @param nodeIdx Node index.
+         * @param uuid Uuid.
+         */
+        public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) {
+            this.subFolderFile = subFolderFile;
+            this.nodeIdx = nodeIdx;
+            this.uuid = uuid;
+        }
+
+        /**
+         * @return Node index (local, usually 0 if multiple nodes are not started at local PC).
+         */
+        public int nodeIndex() {
+            return nodeIdx;
+        }
+
+        /**
+         * @return Uuid contained in file name, is to be set as consistent ID.
+         */
+        public Serializable uuid() {
+            return uuid;
+        }
+
+        /**
+         * @return Absolute file path pointing to DB subfolder within DB storage root folder.
+         */
+        public File subFolderFile() {
+            return subFolderFile;
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
index c47cbc9..28400c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
@@ -19,7 +19,7 @@
 
 import java.io.File;
 import java.io.Serializable;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -28,7 +28,7 @@
 /**
  * Class holds information required for folder generation for ignite persistent store
  */
-public class PdsFolderSettings {
+public class PdsFolderSettings<L extends FileLockHolder> {
     /**
      * DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent store
      * configuration. <br>
@@ -48,7 +48,7 @@
      * directory. This value is to be used at activate instead of locking. <br> May be null in case preconfigured
      * consistent ID is used or in case lock holder was already taken by other processor.
      */
-    @Nullable private final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder;
+    @Nullable private final L fileLockHolder;
 
     /**
      * Indicates if compatible mode is enabled, in that case all sub folders are generated from consistent ID without
@@ -68,7 +68,7 @@
     public PdsFolderSettings(@Nullable final File persistentStoreRootPath,
         final String folderName,
         final Serializable consistentId,
-        @Nullable final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder,
+        @Nullable final L fileLockHolder,
         final boolean compatible) {
 
         this.consistentId = consistentId;
@@ -125,7 +125,7 @@
      *
      * @return File lock holder with prelocked db directory.
      */
-    @Nullable public GridCacheDatabaseSharedManager.FileLockHolder getLockedFileLockHolder() {
+    @Nullable public L getLockedFileLockHolder() {
         return fileLockHolder;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
index cefaa04..6122e28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.filename;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder;
 
 /**
  * Resolves folders for PDS mode, may have side effect as setting random UUID as local node consistent ID.
@@ -29,5 +30,5 @@
      * @return PDS folder settings, consistentID and prelocked DB file lock.
      * @throws IgniteCheckedException if failed.
      */
-    public PdsFolderSettings resolveFolders() throws IgniteCheckedException;
+    public PdsFolderSettings<NodeFileLockHolder> resolveFolders() throws IgniteCheckedException;
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
index 622df61..50837b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
@@ -169,7 +169,7 @@
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirectories;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.DB_DEFAULT_FOLDER;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.DB_DEFAULT_FOLDER;
 import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID;
 import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_NAME;
 import static org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId.getTypeByPartId;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 1732dc2..673accd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -274,6 +274,9 @@
     /** WAL archive directory (including consistent ID as subfolder). */
     private File walArchiveDir;
 
+    /** WAL cdc directory (including consistent ID as subfolder) */
+    private File walCdcDir;
+
     /** Serializer of latest version, used to read header record and for write records */
     private RecordSerializer serializer;
 
@@ -327,6 +330,9 @@
      */
     private final long walAutoArchiveAfterInactivity;
 
+    /** Positive (non-0) value indicates WAL must be archived even if not complete. */
+    private final long walForceArchiveTimeout;
+
     /**
      * Container with last WAL record logged timestamp.<br> Zero value means there was no records logged to current
      * segment, skip possible archiving for this case<br> Value is filled only for case {@link
@@ -334,6 +340,9 @@
      */
     private final AtomicLong lastRecordLoggedMs = new AtomicLong();
 
+    /** Last rollover time. */
+    private AtomicLong lastRolloverMs;
+
     /**
      * Cancellable task for {@link WALMode#BACKGROUND}, should be cancelled at shutdown.
      * Null for non background modes.
@@ -402,8 +411,12 @@
         ioFactory = mode == WALMode.FSYNC ? dsCfg.getFileIOFactory() : new RandomAccessFileIOFactory();
         segmentFileInputFactory = new SimpleSegmentFileInputFactory();
         walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity();
+        walForceArchiveTimeout = dsCfg.getWalForceArchiveTimeout();
 
-        timeoutRolloverMux = walAutoArchiveAfterInactivity > 0 ? new Object() : null;
+        timeoutRolloverMux = (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) ? new Object() : null;
+
+        if (walForceArchiveTimeout > 0)
+            lastRolloverMs = new AtomicLong();
 
         maxWalArchiveSize = dsCfg.getMaxWalArchiveSize();
 
@@ -456,6 +469,15 @@
                 "write ahead log archive directory"
             );
 
+            if (dsCfg.isCdcEnabled()) {
+                walCdcDir = initDirectory(
+                    dsCfg.getCdcWalPath(),
+                    DataStorageConfiguration.DFLT_WAL_CDC_PATH,
+                    resolveFolders.folderName(),
+                    "change data capture directory"
+                );
+            }
+
             serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVer);
 
             GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database();
@@ -748,51 +770,56 @@
         if (mode == WALMode.BACKGROUND)
             backgroundFlushSchedule = cctx.time().schedule(this::doFlush, flushFreq, flushFreq);
 
-        if (walAutoArchiveAfterInactivity > 0)
-            scheduleNextInactivityPeriodElapsedCheck();
+        if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0)
+            scheduleNextRolloverCheck();
     }
 
     /**
-     * Schedules next check of inactivity period expired. Based on current record update timestamp. At timeout method
-     * does check of inactivity period and schedules new launch.
+     * Schedules next rollover check.
+     * If {@link DataStorageConfiguration#getWalForceArchiveTimeout()} configured rollover happens forcefully.
+     * Else check based on current record update timestamp and at timeout method does check of inactivity period and schedules new launch.
      */
-    private void scheduleNextInactivityPeriodElapsedCheck() {
-        assert walAutoArchiveAfterInactivity > 0;
+    private void scheduleNextRolloverCheck() {
+        assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0;
         assert timeoutRolloverMux != null;
 
         synchronized (timeoutRolloverMux) {
-            long lastRecMs = lastRecordLoggedMs.get();
-            long nextEndTime = lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs + walAutoArchiveAfterInactivity;
+            long nextEndTime = walForceArchiveTimeout > 0
+                ? nextTimeout(lastRolloverMs.get(), walForceArchiveTimeout)
+                : nextTimeout(lastRecordLoggedMs.get(), walAutoArchiveAfterInactivity);
 
             cctx.time().addTimeoutObject(timeoutRollover = new TimeoutRollover(nextEndTime));
         }
     }
 
+    /** */
+    private long nextTimeout(long lastEvt, long timeout) {
+        return lastEvt <= 0 ? U.currentTimeMillis() : lastEvt + timeout;
+    }
+
     /** {@inheritDoc} */
     @Override public int serializerVersion() {
         return serializerVer;
     }
 
     /**
-     * Checks if there was elapsed significant period of inactivity. If WAL auto-archive is enabled using
-     * {@link #walAutoArchiveAfterInactivity} > 0 this method will activate roll over by timeout.<br>
+     * Checks if there was elapsed significant period of inactivity or force archive timeout.
+     * If WAL auto-archive is enabled using {@link #walAutoArchiveAfterInactivity} > 0 or {@link #walForceArchiveTimeout}
+     * this method will activate roll over by timeout.
      */
-    private void checkWalRolloverRequiredDuringInactivityPeriod() {
-        if (walAutoArchiveAfterInactivity <= 0)
-            return; // feature not configured, nothing to do
+    private void checkWalRolloverRequired() {
+        if (walAutoArchiveAfterInactivity <= 0 && walForceArchiveTimeout <= 0)
+            return; // feature not configured, nothing to do.
 
-        final long lastRecMs = lastRecordLoggedMs.get();
+        if (lastRecordLoggedMs.get() == 0)
+            return; //no records were logged to current segment, does not consider inactivity.
 
-        if (lastRecMs == 0)
-            return; //no records were logged to current segment, does not consider inactivity
-
-        final long elapsedMs = U.currentTimeMillis() - lastRecMs;
-
-        if (elapsedMs <= walAutoArchiveAfterInactivity)
-            return; // not enough time elapsed since last write
-
-        if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0))
-            return; // record write occurred concurrently
+        if (walForceArchiveTimeout > 0) {
+            if (!checkTimeout(lastRolloverMs, walForceArchiveTimeout))
+                return;
+        }
+        else if (!checkTimeout(lastRecordLoggedMs, walAutoArchiveAfterInactivity))
+            return;
 
         final FileWriteHandle handle = currentHandle();
 
@@ -806,6 +833,19 @@
         }
     }
 
+    /** */
+    private boolean checkTimeout(AtomicLong lastEvt, long timeout) {
+        final long lastEvtMs = lastEvt.get();
+
+        final long elapsedMs = U.currentTimeMillis() - lastEvtMs;
+
+        if (elapsedMs <= timeout)
+            return false; // not enough time elapsed since last write.
+
+        // Will return false if record write occurred concurrently.
+        return lastEvt.compareAndSet(lastEvtMs, 0);
+    }
+
     /** {@inheritDoc} */
     @Override public WALPointer log(WALRecord rec) throws IgniteCheckedException {
         return log(rec, RolloverType.NONE);
@@ -881,7 +921,7 @@
             if (ptr != null) {
                 metrics.onWalRecordLogged(rec.size());
 
-                if (walAutoArchiveAfterInactivity > 0)
+                if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0)
                     lastRecordLoggedMs.set(U.currentTimeMillis());
 
                 return ptr;
@@ -1305,9 +1345,13 @@
 
             assert updated : "Concurrent updates on rollover are not allowed";
 
-            if (walAutoArchiveAfterInactivity > 0)
+            if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) {
                 lastRecordLoggedMs.set(0);
 
+                if (walForceArchiveTimeout > 0)
+                    lastRolloverMs.set(U.currentTimeMillis());
+            }
+
             // Let other threads to proceed with new segment.
             hnd.signalNextAvailable();
         }
@@ -1992,6 +2036,9 @@
 
                 Files.move(dstTmpFile.toPath(), dstFile.toPath());
 
+                if (dsCfg.isCdcEnabled())
+                    Files.createLink(walCdcDir.toPath().resolve(dstFile.getName()), dstFile.toPath());
+
                 if (mode != WALMode.NONE) {
                     try (FileIO f0 = ioFactory.create(dstFile, CREATE, READ, WRITE)) {
                         f0.force();
@@ -3443,7 +3490,7 @@
 
         /** {@inheritDoc} */
         @Override public void onTimeout() {
-            assert walAutoArchiveAfterInactivity > 0;
+            assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0;
             assert timeoutRolloverMux != null;
 
             synchronized (timeoutRolloverMux) {
@@ -3453,9 +3500,9 @@
                             new Time(U.currentTimeMillis()).toString() + ")");
                     }
 
-                    checkWalRolloverRequiredDuringInactivityPeriod();
+                    checkWalRolloverRequired();
 
-                    scheduleNextInactivityPeriodElapsedCheck();
+                    scheduleNextRolloverCheck();
                 }
             }
         }
@@ -3464,7 +3511,7 @@
          * Cancel auto rollover.
          */
         public void cancel() {
-            assert walAutoArchiveAfterInactivity > 0;
+            assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0;
             assert timeoutRolloverMux != null;
 
             synchronized (timeoutRolloverMux) {
@@ -3480,7 +3527,7 @@
      * Stop auto rollover.
      */
     private void stopAutoRollover() {
-        if (walAutoArchiveAfterInactivity > 0) {
+        if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) {
             assert timeoutRolloverMux != null;
 
             synchronized (timeoutRolloverMux) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index db46d6f..e7c18ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -358,7 +358,9 @@
         GridKernalContext kernalCtx = sharedCtx.kernalContext();
         IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects();
 
-        if (processor != null && (rec.type() == RecordType.DATA_RECORD || rec.type() == RecordType.MVCC_DATA_RECORD)) {
+        if (processor != null && (rec.type() == RecordType.DATA_RECORD
+            || rec.type() == RecordType.DATA_RECORD_V2
+            || rec.type() == RecordType.MVCC_DATA_RECORD)) {
             try {
                 return postProcessDataRecord((DataRecord)rec, kernalCtx, processor);
             }
@@ -502,7 +504,8 @@
                 dataEntry.partitionId(),
                 dataEntry.partitionCounter(),
                 coCtx,
-                keepBinary);
+                keepBinary,
+                dataEntry.flags());
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index bc5f9e3..ff4a199 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -43,6 +43,7 @@
 import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecordV2;
 import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord;
 import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord;
+import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
 import org.apache.ignite.internal.pagemem.wal.record.ReencryptionStartRecord;
 import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
@@ -121,7 +122,9 @@
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V3;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD_V2;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.MASTER_KEY_CHANGE_RECORD_V2;
@@ -406,7 +409,7 @@
             case PARTITION_DESTROY:
                 return /*cacheId*/4 + /*partId*/4;
 
-            case DATA_RECORD:
+            case DATA_RECORD_V2:
                 DataRecord dataRec = (DataRecord)record;
 
                 return 4 + dataSize(dataRec);
@@ -667,12 +670,13 @@
                 break;
 
             case DATA_RECORD:
+            case DATA_RECORD_V2:
                 int entryCnt = in.readInt();
 
                 List<DataEntry> entries = new ArrayList<>(entryCnt);
 
                 for (int i = 0; i < entryCnt; i++)
-                    entries.add(readPlainDataEntry(in));
+                    entries.add(readPlainDataEntry(in, type));
 
                 res = new DataRecord(entries, 0L);
 
@@ -680,12 +684,13 @@
 
             case ENCRYPTED_DATA_RECORD:
             case ENCRYPTED_DATA_RECORD_V2:
+            case ENCRYPTED_DATA_RECORD_V3:
                 entryCnt = in.readInt();
 
                 entries = new ArrayList<>(entryCnt);
 
                 for (int i = 0; i < entryCnt; i++)
-                    entries.add(readEncryptedDataEntry(in, type == ENCRYPTED_DATA_RECORD_V2));
+                    entries.add(readEncryptedDataEntry(in, type));
 
                 res = new DataRecord(entries, 0L);
 
@@ -1351,7 +1356,7 @@
 
                 break;
 
-            case DATA_RECORD:
+            case DATA_RECORD_V2:
                 DataRecord dataRec = (DataRecord)rec;
 
                 buf.putInt(dataRec.writeEntries().size());
@@ -1956,6 +1961,9 @@
         buf.putInt(entry.partitionId());
         buf.putLong(entry.partitionCounter());
         buf.putLong(entry.expireTime());
+
+        if (!(entry instanceof MvccDataEntry))
+            buf.put(entry.flags());
     }
 
     /**
@@ -2002,14 +2010,16 @@
 
     /**
      * @param in Input to read from.
-     * @param readKeyId If {@code true} encryption key identifier will be read from {@code in}.
+     * @param recType Record type.
      * @return Read entry.
      * @throws IOException If failed.
      * @throws IgniteCheckedException If failed.
      */
-    DataEntry readEncryptedDataEntry(ByteBufferBackedDataInput in, boolean readKeyId) throws IOException, IgniteCheckedException {
+    DataEntry readEncryptedDataEntry(ByteBufferBackedDataInput in, RecordType recType) throws IOException, IgniteCheckedException {
         boolean needDecryption = in.readByte() == ENCRYPTED;
 
+        RecordType dataRecordType = recType == ENCRYPTED_DATA_RECORD_V3 ? DATA_RECORD_V2 : DATA_RECORD;
+
         if (needDecryption) {
             if (encSpi == null) {
                 skipEncryptedRecord(in, false);
@@ -2017,22 +2027,23 @@
                 return new EncryptedDataEntry();
             }
 
-            T3<ByteBufferBackedDataInput, Integer, RecordType> clData = readEncryptedData(in, false, readKeyId);
+            T3<ByteBufferBackedDataInput, Integer, RecordType> clData = readEncryptedData(in, false,
+                recType == ENCRYPTED_DATA_RECORD_V2 || recType == ENCRYPTED_DATA_RECORD_V3);
 
             if (clData.get1() == null)
                 return null;
 
-            return readPlainDataEntry(clData.get1());
+            return readPlainDataEntry(clData.get1(), dataRecordType);
         }
 
-        return readPlainDataEntry(in);
+        return readPlainDataEntry(in, dataRecordType);
     }
 
     /**
      * @param in Input to read from.
      * @return Read entry.
      */
-    DataEntry readPlainDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
+    DataEntry readPlainDataEntry(ByteBufferBackedDataInput in, RecordType type) throws IOException, IgniteCheckedException {
         int cacheId = in.readInt();
 
         int keySize = in.readInt();
@@ -2061,6 +2072,7 @@
         int partId = in.readInt();
         long partCntr = in.readLong();
         long expireTime = in.readLong();
+        byte flags = type == DATA_RECORD_V2 ? in.readByte() : (byte)0;
 
         GridCacheContext cacheCtx = cctx.cacheContext(cacheId);
 
@@ -2083,7 +2095,8 @@
                     writeVer,
                     expireTime,
                     partId,
-                    partCntr
+                    partCntr,
+                    flags
             );
         }
         else
@@ -2099,7 +2112,9 @@
                     writeVer,
                     expireTime,
                     partId,
-                    partCntr);
+                    partCntr,
+                    flags
+            );
     }
 
     /**
@@ -2113,10 +2128,10 @@
         if (needEncryption(rec))
             return ENCRYPTED_RECORD_V2;
 
-        if (rec.type() != DATA_RECORD)
+        if (rec.type() != DATA_RECORD && rec.type() != DATA_RECORD_V2)
             return rec.type();
 
-        return isDataRecordEncrypted((DataRecord)rec) ? ENCRYPTED_DATA_RECORD_V2 : DATA_RECORD;
+        return isDataRecordEncrypted((DataRecord)rec) ? ENCRYPTED_DATA_RECORD_V3 : rec.type();
     }
 
     /**
@@ -2235,7 +2250,8 @@
             /*write ver*/CacheVersionIO.size(entry.writeVersion(), false) +
             /*part ID*/4 +
             /*expire Time*/8 +
-            /*part cnt*/8;
+            /*part cnt*/8 +
+            /*primary*/(entry instanceof MvccDataEntry ? 0 : 1);
     }
 
     /**
@@ -2268,7 +2284,7 @@
     public static class EncryptedDataEntry extends DataEntry {
         /** Constructor. */
         EncryptedDataEntry() {
-            super(0, null, null, READ, null, null, 0, 0, 0);
+            super(0, null, null, READ, null, null, 0, 0, 0, EMPTY_FLAGS);
         }
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
index 8622629..56568cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
@@ -56,8 +56,6 @@
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
-import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2;
-
 /**
  * Record data V2 serializer.
  */
@@ -95,7 +93,7 @@
             case MVCC_DATA_RECORD:
                 return 4/*entry count*/ + 8/*timestamp*/ + dataSize((DataRecord)rec);
 
-            case DATA_RECORD:
+            case DATA_RECORD_V2:
                 return super.plainSize(rec) + 8/*timestamp*/;
 
             case SNAPSHOT:
@@ -160,13 +158,14 @@
                 return cpRec;
 
             case DATA_RECORD:
+            case DATA_RECORD_V2:
                 int entryCnt = in.readInt();
                 long timeStamp = in.readLong();
 
                 List<DataEntry> entries = new ArrayList<>(entryCnt);
 
                 for (int i = 0; i < entryCnt; i++)
-                    entries.add(readPlainDataEntry(in));
+                    entries.add(readPlainDataEntry(in, type));
 
                 return new DataRecord(entries, timeStamp);
 
@@ -183,13 +182,14 @@
 
             case ENCRYPTED_DATA_RECORD:
             case ENCRYPTED_DATA_RECORD_V2:
+            case ENCRYPTED_DATA_RECORD_V3:
                 entryCnt = in.readInt();
                 timeStamp = in.readLong();
 
                 entries = new ArrayList<>(entryCnt);
 
                 for (int i = 0; i < entryCnt; i++)
-                    entries.add(readEncryptedDataEntry(in, type == ENCRYPTED_DATA_RECORD_V2));
+                    entries.add(readEncryptedDataEntry(in, type));
 
                 return new DataRecord(entries, timeStamp);
 
@@ -260,7 +260,7 @@
                 break;
 
             case MVCC_DATA_RECORD:
-            case DATA_RECORD:
+            case DATA_RECORD_V2:
                 DataRecord dataRec = (DataRecord)rec;
 
                 buf.putInt(dataRec.writeEntries().size());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 9a3da6e..3c69289 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -862,7 +862,8 @@
                                             writeVersion(),
                                             0,
                                             txEntry.key().partition(),
-                                            txEntry.updateCounter())));
+                                            txEntry.updateCounter(),
+                                            DataEntry.flags(CU.txOnPrimary(this)))));
                                     }
 
                                     ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
index 4f4f9d0..7769501 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
@@ -23,6 +23,7 @@
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import org.apache.ignite.cache.CacheEntryVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -31,7 +32,7 @@
 /**
  * Grid unique version.
  */
-public class GridCacheVersion implements Message, Comparable<GridCacheVersion>, Externalizable {
+public class GridCacheVersion implements Message, Externalizable, CacheEntryVersion {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -92,10 +93,8 @@
         this.order = order;
     }
 
-    /**
-     * @return Topology version plus number of seconds from the start time of the first grid node..
-     */
-    public int topologyVersion() {
+    /** {@inheritDoc} */
+    @Override public int topologyVersion() {
         return topVer;
     }
 
@@ -111,17 +110,25 @@
     /**
      * @return Version order.
      */
-    public long order() {
+    @Override public long order() {
         return order;
     }
 
-    /**
-     * @return Node order on which this version was assigned.
-     */
-    public int nodeOrder() {
+    /** {@inheritDoc} */
+    @Override public CacheEntryVersion otherClusterVersion() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int nodeOrder() {
         return nodeOrderDrId & NODE_ORDER_MASK;
     }
 
+    /** {@inheritDoc} */
+    @Override public byte clusterId() {
+        return dataCenterId();
+    }
+
     /**
      * @return DR mask.
      */
@@ -219,13 +226,13 @@
     }
 
     /** {@inheritDoc} */
-    @Override public int compareTo(GridCacheVersion other) {
+    @Override public int compareTo(CacheEntryVersion other) {
         int res = Integer.compare(topologyVersion(), other.topologyVersion());
 
         if (res != 0)
             return res;
 
-        res = Long.compare(order, other.order);
+        res = Long.compare(order(), other.order());
 
         if (res != 0)
             return res;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java
index 5eb8c7c..1d5bad0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java
@@ -22,6 +22,7 @@
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
+import org.apache.ignite.cache.CacheEntryVersion;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 
@@ -82,6 +83,11 @@
     }
 
     /** {@inheritDoc} */
+    @Override public CacheEntryVersion otherClusterVersion() {
+        return conflictVersion();
+    }
+
+    /** {@inheritDoc} */
     @Override public short directType() {
         return 104;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index e389662..fb67160 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -136,6 +136,11 @@
         startVer = new GridCacheVersion(0, 0, 0, dataCenterId);
     }
 
+    /** @return Data center ID. */
+    public byte dataCenterId() {
+        return dataCenterId;
+    }
+
     /**
      * @param nodeId Node ID.
      * @param ver Remote version.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index ab785f1..660786b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -2330,7 +2330,8 @@
                             false,
                             topVer,
                             primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD,
-                            false);
+                            false,
+                            primary);
 
                         entry.touch();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 5bead46..9ac69d32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -2063,7 +2063,8 @@
                 .setSystemRegionMaxSize(in.readLong())
                 .setPageSize(in.readInt())
                 .setConcurrencyLevel(in.readInt())
-                .setWalAutoArchiveAfterInactivity(in.readLong());
+                .setWalAutoArchiveAfterInactivity(in.readLong())
+                .setWalForceArchiveTimeout(in.readLong());
 
         if (in.readBoolean())
             res.setCheckpointReadLockTimeout(in.readLong());
@@ -2199,6 +2200,7 @@
             w.writeInt(cfg.getPageSize());
             w.writeInt(cfg.getConcurrencyLevel());
             w.writeLong(cfg.getWalAutoArchiveAfterInactivity());
+            w.writeLong(cfg.getWalForceArchiveTimeout());
 
             if (cfg.getCheckpointReadLockTimeout() != null) {
                 w.writeBoolean(true);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
index 4b43e04..9bc47fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
@@ -106,7 +106,7 @@
      * @return {@code True} if resource was injected.
      * @throws IgniteCheckedException Thrown in case of any errors during injection.
      */
-    boolean inject(Object target,
+    public boolean inject(Object target,
         Class<? extends Annotation> annCls,
         GridResourceInjector injector,
         @Nullable GridDeployment dep,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 3ddea5d..069ca37 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -256,6 +256,9 @@
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
+import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.java.JavaLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -4637,6 +4640,130 @@
     }
 
     /**
+     * Resolves work directory.
+     * @param cfg Ignite configuration.
+     */
+    public static void initWorkDir(IgniteConfiguration cfg) throws IgniteCheckedException {
+        String igniteHome = cfg.getIgniteHome();
+
+        // Set Ignite home.
+        if (igniteHome == null)
+            igniteHome = U.getIgniteHome();
+
+        String userProvidedWorkDir = cfg.getWorkDirectory();
+
+        // Correctly resolve work directory and set it back to configuration.
+        cfg.setWorkDirectory(U.workDirectory(userProvidedWorkDir, igniteHome));
+    }
+
+    /**
+     * @param cfg Ignite configuration.
+     * @param app Application name.
+     * @return Initialized logger.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static IgniteLogger initLogger(IgniteConfiguration cfg, String app) throws IgniteCheckedException {
+        return initLogger(
+            cfg.getGridLogger(),
+            app,
+            cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID(),
+            cfg.getWorkDirectory()
+        );
+    }
+
+    /**
+     * @param cfgLog Configured logger.
+     * @param app Application name.
+     * @param workDir Work directory.
+     * @return Initialized logger.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("ErrorNotRethrown")
+    public static IgniteLogger initLogger(
+        @Nullable IgniteLogger cfgLog,
+        @Nullable String app,
+        UUID nodeId,
+        String workDir
+    ) throws IgniteCheckedException {
+        try {
+            Exception log4jInitErr = null;
+
+            if (cfgLog == null) {
+                Class<?> log4jCls;
+
+                try {
+                    log4jCls = Class.forName("org.apache.ignite.logger.log4j.Log4JLogger");
+                }
+                catch (ClassNotFoundException | NoClassDefFoundError ignored) {
+                    log4jCls = null;
+                }
+
+                if (log4jCls != null) {
+                    try {
+                        URL url = U.resolveIgniteUrl("config/ignite-log4j.xml");
+
+                        if (url == null) {
+                            File cfgFile = new File("config/ignite-log4j.xml");
+
+                            if (!cfgFile.exists())
+                                cfgFile = new File("../config/ignite-log4j.xml");
+
+                            if (cfgFile.exists()) {
+                                try {
+                                    url = cfgFile.toURI().toURL();
+                                }
+                                catch (MalformedURLException ignore) {
+                                    // No-op.
+                                }
+                            }
+                        }
+
+                        if (url != null) {
+                            boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null);
+
+                            if (configured)
+                                url = null;
+                        }
+
+                        if (url != null) {
+                            Constructor<?> ctor = log4jCls.getConstructor(URL.class);
+
+                            cfgLog = (IgniteLogger)ctor.newInstance(url);
+                        }
+                        else
+                            cfgLog = (IgniteLogger)log4jCls.newInstance();
+                    }
+                    catch (Exception e) {
+                        log4jInitErr = e;
+                    }
+                }
+
+                if (log4jCls == null || log4jInitErr != null)
+                    cfgLog = new JavaLogger();
+            }
+
+            // Special handling for Java logger which requires work directory.
+            if (cfgLog instanceof JavaLogger)
+                ((JavaLogger)cfgLog).setWorkDirectory(workDir);
+
+            // Set node IDs for all file appenders.
+            if (cfgLog instanceof LoggerNodeIdAndApplicationAware)
+                ((LoggerNodeIdAndApplicationAware)cfgLog).setApplicationAndNode(app, nodeId);
+            else if (cfgLog instanceof LoggerNodeIdAware)
+                ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId);
+
+            if (log4jInitErr != null)
+                U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): "
+                    + log4jInitErr.getCause());
+
+            return cfgLog;
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to create logger.", e);
+        }
+    }
+
+    /**
      * Depending on whether or not log is provided and quiet mode is enabled logs given
      * messages as quiet message or normal log INF0 message.
      * <p>
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java
index 2424b2e..d77f3b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java
@@ -53,6 +53,9 @@
     /** Version 4. */
     protected static final byte V4 = 4;
 
+    /** Version 5. */
+    protected static final byte V5 = 5;
+
     /**
      * @param col Source collection.
      * @param <T> Collection type.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
index d0e0f91..160b0fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
@@ -116,9 +116,12 @@
     /** Time interval (in milliseconds) for rate-based metrics. */
     private long metricsRateTimeInterval;
 
-    /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment */
+    /** Time interval of inactivity (in milliseconds) for running auto archiving for incompletely WAL segment. */
     private long walAutoArchiveAfterInactivity;
 
+    /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment. */
+    private long walForceArchiveTimeout;
+
     /** If true, threads that generate dirty pages too fast during ongoing checkpoint will be throttled. */
     private boolean writeThrottlingEnabled;
 
@@ -177,6 +180,7 @@
         metricsSubIntervalCount = cfg.getMetricsSubIntervalCount();
         metricsRateTimeInterval = cfg.getMetricsRateTimeInterval();
         walAutoArchiveAfterInactivity = cfg.getWalAutoArchiveAfterInactivity();
+        walForceArchiveTimeout = cfg.getWalForceArchiveTimeout();
         writeThrottlingEnabled = cfg.isWriteThrottlingEnabled();
         walCompactionEnabled = cfg.isWalCompactionEnabled();
     }
@@ -371,6 +375,13 @@
     }
 
     /**
+     * @return Time in millis.
+     */
+    public long getWalForceArchiveTimeout() {
+        return walForceArchiveTimeout;
+    }
+
+    /**
      * @return Flag indicating whether write throttling is enabled.
      */
     public boolean isWriteThrottlingEnabled() {
@@ -428,6 +439,7 @@
         out.writeBoolean(writeThrottlingEnabled);
         out.writeInt(walBufSize);
         out.writeBoolean(walCompactionEnabled);
+        out.writeLong(walForceArchiveTimeout);
     }
 
     /** {@inheritDoc} */
@@ -466,6 +478,9 @@
             walBufSize = in.readInt();
             walCompactionEnabled = in.readBoolean();
         }
+
+        if (protoVer >= V5)
+            walForceArchiveTimeout = in.readLong();
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
index 13c0e72..0f5abc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
@@ -390,7 +390,7 @@
 
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V4;
+        return V5;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
index dffe840..6f506ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
@@ -373,7 +373,7 @@
 
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V4;
+        return V5;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java
index 399e5c2..20ed194 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java
@@ -144,7 +144,7 @@
 
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V4;
+        return V5;
     }
 
     /** */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
index c474387..579eabe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java
@@ -117,7 +117,7 @@
 
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V4;
+        return V5;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java
new file mode 100644
index 0000000..2b2ba1e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java
@@ -0,0 +1,39 @@
+/*
+ * 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.logger;
+
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for Ignite file appenders to attach postfix to log file names.
+ */
+public interface LoggerNodeIdAndApplicationAware extends LoggerNodeIdAware {
+    /** {@inheritDoc} */
+    @Override public default void setNodeId(UUID nodeId) {
+        setApplicationAndNode(null, nodeId);
+    }
+
+    /**
+     * Sets application name and node ID.
+     *
+     * @param application Application.
+     * @param nodeId Node ID.
+     */
+    public void setApplicationAndNode(@Nullable String application, UUID nodeId);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java
index 492d93c..a17e340 100644
--- a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java
+++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java
@@ -21,7 +21,10 @@
 
 /**
  * Interface for Ignite file appenders to attach node ID to log file names.
+ *
+ * @deprecated Use {@link LoggerNodeIdAndApplicationAware} instead.
  */
+@Deprecated
 public interface LoggerNodeIdAware {
     /**
      * Sets node ID.
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
index c82f01d..b7c36e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
@@ -34,7 +34,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.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.logging.Level.FINE;
@@ -95,7 +95,7 @@
  * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger
  * injection.
  */
-public class JavaLogger implements IgniteLogger, LoggerNodeIdAware {
+public class JavaLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware {
     /** */
     public static final String DFLT_CONFIG_PATH = "config/java.util.logging.properties";
 
@@ -365,7 +365,7 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void setNodeId(UUID nodeId) {
+    @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) {
         A.notNull(nodeId, "nodeId");
 
         if (this.nodeId != null)
@@ -385,7 +385,7 @@
             return;
 
         try {
-            fileHnd.nodeId(nodeId, workDir);
+            fileHnd.nodeId(application, nodeId, workDir);
         }
         catch (IgniteCheckedException | IOException e) {
             throw new RuntimeException("Failed to enable file handler.", e);
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java
index 6320589..5560e43 100644
--- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java
@@ -80,6 +80,16 @@
      * @param nodeId Node id.
      */
     public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, IOException {
+        nodeId(null, nodeId, workDir);
+    }
+
+    /**
+     * Sets Node id and instantiates {@link FileHandler} delegate.
+     *
+     * @param app Application name.
+     * @param nodeId Node id.
+     */
+    public void nodeId(@Nullable String app, UUID nodeId, String workDir) throws IgniteCheckedException, IOException {
         if (delegate != null)
             return;
 
@@ -88,9 +98,13 @@
         String ptrn = manager.getProperty(clsName + ".pattern");
 
         if (ptrn == null)
-            ptrn = "ignite-%{id8}.%g.log";
+            ptrn = "%{app}-%{id8}.%g.log";
 
-        ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath();
+        String fileName = ptrn
+            .replace("%{app}", app != null ? app : "ignite")
+            .replace("%{id8}", U.id8(nodeId));
+
+        ptrn = new File(logDirectory(workDir), fileName).getAbsolutePath();
 
         int limit = getIntProperty(clsName + ".limit", 0);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java
new file mode 100644
index 0000000..da5c84f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java
@@ -0,0 +1,164 @@
+/*
+ * 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.startup.cmdline;
+
+import java.net.URL;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cdc.CdcConfiguration;
+import org.apache.ignite.cdc.CdcLoader;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
+import org.apache.ignite.internal.util.typedef.X;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_SHUTDOWN_HOOK;
+import static org.apache.ignite.internal.IgniteKernal.NL;
+import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR;
+import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT;
+import static org.apache.ignite.startup.cmdline.CommandLineStartup.isHelp;
+
+/**
+ * This class defines command-line Ignite Capture Data Change startup. This startup can be used to start Ignite
+ * Capture Data Change application outside of any hosting environment from command line.
+ * This startup is a Java application with {@link #main(String[])} method that accepts command line arguments.
+ * It accepts just one parameter which is Spring XML configuration file path.
+ * You can run this class from command line without parameters to get help message.
+ * <p>
+ * Note that scripts {@code ${IGNITE_HOME}/bin/cdc.{sh|bat}} shipped with Ignite use
+ * this startup and you can use them as an example.
+ * <p>
+ *
+ * @see CdcMain
+ */
+public class CdcCommandLineStartup {
+    /** Quite log flag. */
+    private static final boolean QUITE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_QUIET);
+
+    /**
+     * Main entry point.
+     *
+     * @param args Command line arguments.
+     */
+    public static void main(String[] args) {
+        if (!QUITE) {
+            X.println("    __________  ________________    ________  _____" + NL +
+                      "   /  _/ ___/ |/ /  _/_  __/ __/   / ___/ _ \\/ ___/" + NL +
+                      "  _/ // (7 7    // /  / / / _/    / /__/ // / /__  " + NL +
+                      " /___/\\___/_/|_/___/ /_/ /___/    \\___/____/\\___/");
+            X.println("Ignite Change Data Capture Command Line Startup, ver. " + ACK_VER_STR);
+            X.println(COPYRIGHT);
+            X.println();
+        }
+
+        if (args.length > 1)
+            exit("Too many arguments.", true, -1);
+
+        if (args.length > 0 && isHelp(args[0]))
+            exit(null, true, 0);
+
+        if (args.length > 0 && args[0].isEmpty())
+            exit("Empty argument.", true, 1);
+
+        if (args.length > 0 && args[0].charAt(0) == '-')
+            exit("Invalid arguments: " + args[0], true, -1);
+
+        AtomicReference<CdcMain> cdc = new AtomicReference<>();
+
+        try {
+            cdc.set(CdcLoader.loadCdc(args[0]));
+
+            if (!IgniteSystemProperties.getBoolean(IGNITE_NO_SHUTDOWN_HOOK, false)) {
+                Runtime.getRuntime().addShutdownHook(new Thread("cdc-shutdown-hook") {
+                    @Override public void run() {
+                        cdc.get().stop();
+                    }
+                });
+            }
+
+            Thread appThread = new Thread(cdc.get());
+
+            appThread.start();
+
+            appThread.join();
+        }
+        catch (InterruptedException e) {
+            if (cdc.get() != null)
+                cdc.get().stop();
+        }
+        catch (Throwable e) {
+            e.printStackTrace();
+
+            String note = "";
+
+            if (X.hasCause(e, ClassNotFoundException.class))
+                note = "\nNote! You may use 'USER_LIBS' environment variable to specify your classpath.";
+
+            exit("Failed to run CDC: " + e.getMessage() + note, false, -1);
+        }
+    }
+
+    /**
+     * @param cfgUrl String configuration URL.
+     * @param spring Ignite spring helper.
+     * @return CDC consumer defined in spring configuration.
+     * @throws IgniteCheckedException in case of load error.
+     */
+    private static CdcConfiguration consumerConfig(
+        URL cfgUrl,
+        IgniteSpringHelper spring
+    ) throws IgniteCheckedException {
+        Map<Class<?>, Object> cdcCfgs = spring.loadBeans(cfgUrl, CdcConfiguration.class);
+
+        if (cdcCfgs == null || cdcCfgs.size() != 1)
+            exit("Exact 1 CaptureDataChangeConfiguration configuration should be defined", false, 1);
+
+        return (CdcConfiguration)cdcCfgs.values().iterator().next();
+    }
+
+    /**
+     * Exists with optional error message, usage show and exit code.
+     *
+     * @param errMsg Optional error message.
+     * @param showUsage Whether or not to show usage information.
+     * @param exitCode Exit code.
+     */
+    private static void exit(@Nullable String errMsg, boolean showUsage, int exitCode) {
+        if (errMsg != null)
+            X.error(errMsg);
+
+        if (showUsage) {
+            X.error(
+                "Usage:",
+                "    ignite-cdc.{sh|bat} [?]|[path]",
+                "    Where:",
+                "    ?, /help, -help, - show this message.",
+                "    -v               - verbose mode (quiet by default).",
+                "    path            - path to Spring XML configuration file.",
+                "                      Path can be absolute or relative to IGNITE_HOME.",
+                " ",
+                "Spring file should contain bean definition of 'org.apache.ignite.configuration.IgniteConfiguration' " +
+                "And one or more implementations of 'org.apache.ignite.cdc.CDCConsumer'." +
+                "Note that bean will be fetched by the type and its ID is not used.");
+        }
+
+        System.exit(exitCode);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java
index c12365b..460f7c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java
+++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java
@@ -255,7 +255,7 @@
      * @param arg Command line argument.
      * @return {@code true} if given argument is a help argument, {@code false} otherwise.
      */
-    private static boolean isHelp(String arg) {
+    public static boolean isHelp(String arg) {
         String s;
 
         if (arg.startsWith("--"))
diff --git a/modules/core/src/test/config/log4j2-test.xml b/modules/core/src/test/config/log4j2-test.xml
index 8b46f00..9740c2c 100644
--- a/modules/core/src/test/config/log4j2-test.xml
+++ b/modules/core/src/test/config/log4j2-test.xml
@@ -31,8 +31,8 @@
         <Routing name="FILE">
             <Routes pattern="$${sys:nodeId}">
                 <Route>
-                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}.log"
-                                 filePattern="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
+                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}.log"
+                                 filePattern="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
                         <PatternLayout pattern="[%d{ISO8601}][%-5p][%t][%c{1}]%notEmpty{[%markerSimpleName]} %m%n"/>
                         <Policies>
                             <TimeBasedTriggeringPolicy interval="6" modulate="true" />
diff --git a/modules/core/src/test/config/log4j2-verbose-test.xml b/modules/core/src/test/config/log4j2-verbose-test.xml
index ff6325b..d2d7a780 100644
--- a/modules/core/src/test/config/log4j2-verbose-test.xml
+++ b/modules/core/src/test/config/log4j2-verbose-test.xml
@@ -26,8 +26,8 @@
         <Routing name="FILE">
             <Routes pattern="$${sys:nodeId}">
                 <Route>
-                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}.log"
-                                 filePattern="${sys:IGNITE_HOME}/work/log/ignite-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
+                    <RollingFile name="Rolling-${sys:nodeId}" fileName="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}.log"
+                                 filePattern="${sys:IGNITE_HOME}/work/log/${sys:appId}-${sys:nodeId}-%i-%d{yyyy-MM-dd}.log.gz">
                         <PatternLayout pattern="[%d{ISO8601}][%-5p][%t][%c{1}]%notEmpty{[%markerSimpleName]} %m%n"/>
                         <Policies>
                             <TimeBasedTriggeringPolicy interval="6" modulate="true" />
diff --git a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java
index 17ff241..c9282b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java
@@ -82,7 +82,8 @@
                 ver,
                 0L,
                 partId,
-                updateCntr
+                updateCntr,
+                DataEntry.EMPTY_FLAGS
             );
 
             IgniteCacheDatabaseSharedManager db = ctx.shared().database();
diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java
new file mode 100644
index 0000000..e250f68
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java
@@ -0,0 +1,252 @@
+/*
+ * 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.cdc;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.util.typedef.CI3;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE;
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/** */
+public abstract class AbstractCdcTest extends GridCommonAbstractTest {
+    /** */
+    public static final String JOHN = "John Connor";
+
+    /** */
+    public static final int WAL_ARCHIVE_TIMEOUT = 5_000;
+
+    /** Keys count. */
+    public static final int KEYS_CNT = 50;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.beforeTest();
+    }
+
+    /** */
+    public void addAndWaitForConsumption(
+        UserCdcConsumer cnsmr,
+        CdcMain cdc,
+        IgniteCache<Integer, CdcSelfTest.User> cache,
+        IgniteCache<Integer, CdcSelfTest.User> txCache,
+        CI3<IgniteCache<Integer, CdcSelfTest.User>, Integer, Integer> addData,
+        int from,
+        int to,
+        long timeout
+    ) throws IgniteCheckedException {
+        IgniteInternalFuture<?> fut = runAsync(cdc);
+
+        addData.apply(cache, from, to);
+
+        if (txCache != null)
+            addData.apply(txCache, from, to);
+
+        assertTrue(waitForSize(to - from, cache.getName(), UPDATE, timeout, cnsmr));
+
+        if (txCache != null)
+            assertTrue(waitForSize(to - from, txCache.getName(), UPDATE, timeout, cnsmr));
+
+        fut.cancel();
+
+        List<Integer> keys = cnsmr.data(UPDATE, cacheId(cache.getName()));
+
+        assertEquals(to - from, keys.size());
+
+        for (int i = from; i < to; i++)
+            assertTrue(Integer.toString(i), keys.contains(i));
+
+        assertTrue(cnsmr.stopped());
+    }
+
+    /** */
+    public boolean waitForSize(
+        int expSz,
+        String cacheName,
+        CdcSelfTest.ChangeEventType evtType,
+        long timeout,
+        TestCdcConsumer<?>... cnsmrs
+    ) throws IgniteInterruptedCheckedException {
+        return waitForCondition(
+            () -> {
+                int sum = Arrays.stream(cnsmrs).mapToInt(c -> F.size(c.data(evtType, cacheId(cacheName)))).sum();
+                return sum == expSz;
+            },
+            timeout);
+    }
+
+    /** */
+    public CdcConfiguration cdcConfig(CdcConsumer cnsmr) {
+        CdcConfiguration cdcCfg = new CdcConfiguration();
+
+        cdcCfg.setConsumer(cnsmr);
+        cdcCfg.setKeepBinary(false);
+
+        return cdcCfg;
+    }
+
+    /** */
+    public abstract static class TestCdcConsumer<T> implements CdcConsumer {
+        /** Keys */
+        final ConcurrentMap<IgniteBiTuple<ChangeEventType, Integer>, List<T>> data = new ConcurrentHashMap<>();
+
+        /** */
+        private volatile boolean stopped;
+
+        /** {@inheritDoc} */
+        @Override public void start() {
+            stopped = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() {
+            stopped = true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+            evts.forEachRemaining(evt -> {
+                if (!evt.primary())
+                    return;
+
+                data.computeIfAbsent(
+                    F.t(evt.value() == null ? DELETE : UPDATE, evt.cacheId()),
+                    k -> new ArrayList<>()).add(extract(evt));
+
+                checkEvent(evt);
+            });
+
+            return commit();
+        }
+
+        /** */
+        public abstract void checkEvent(CdcEvent evt);
+
+        /** */
+        public abstract T extract(CdcEvent evt);
+
+        /** */
+        protected boolean commit() {
+            return true;
+        }
+
+        /** @return Read keys. */
+        public List<T> data(ChangeEventType op, int cacheId) {
+            return data.get(F.t(op, cacheId));
+        }
+
+        /** */
+        public boolean stopped() {
+            return stopped;
+        }
+    }
+
+    /** */
+    public static class UserCdcConsumer extends TestCdcConsumer<Integer> {
+        /** {@inheritDoc} */
+        @Override public void checkEvent(CdcEvent evt) {
+            assertNull(evt.version().otherClusterVersion());
+
+            if (evt.value() == null)
+                return;
+
+            User user = (User)evt.value();
+
+            assertTrue(user.getName().startsWith(JOHN));
+            assertTrue(user.getAge() >= 42);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer extract(CdcEvent evt) {
+            return (Integer)evt.key();
+        }
+    }
+
+    /** */
+    protected static User createUser(int i) {
+        byte[] bytes = new byte[1024];
+
+        ThreadLocalRandom.current().nextBytes(bytes);
+
+        return new User(JOHN + " " + i, 42 + i, bytes);
+    }
+
+    /** */
+    public static class User {
+        /** */
+        private final String name;
+
+        /** */
+        private final int age;
+
+        /** */
+        private final byte[] payload;
+
+        /** */
+        public User(String name, int age, byte[] payload) {
+            this.name = name;
+            this.age = age;
+            this.payload = payload;
+        }
+
+        /** */
+        public String getName() {
+            return name;
+        }
+
+        /** */
+        public int getAge() {
+            return age;
+        }
+
+        /** */
+        public byte[] getPayload() {
+            return payload;
+        }
+    }
+
+    /** */
+    public enum ChangeEventType {
+        /** */
+        UPDATE,
+
+        /** */
+        DELETE
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java
new file mode 100644
index 0000000..30fd1e9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java
@@ -0,0 +1,230 @@
+/*
+ * 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.cdc;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+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.IgniteInternalFuture;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
+import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.plugin.AbstractCachePluginProvider;
+import org.apache.ignite.plugin.AbstractTestPluginProvider;
+import org.apache.ignite.plugin.CachePluginContext;
+import org.apache.ignite.plugin.CachePluginProvider;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/** */
+public class CdcCacheVersionTest extends AbstractCdcTest {
+    /** */
+    public static final String FOR_OTHER_CLUSTER_ID = "for-other-cluster-id";
+
+    /** */
+    public static final byte DFLT_CLUSTER_ID = 1;
+
+    /** */
+    public static final byte OTHER_CLUSTER_ID = 2;
+
+    /** */
+    public static final int KEY_TO_UPD = 42;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setCdcEnabled(true)
+            .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT)
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true)));
+
+        cfg.setPluginProviders(new AbstractTestPluginProvider() {
+            @Override public String name() {
+                return "ConflictResolverProvider";
+            }
+
+            @Override public CachePluginProvider createCacheProvider(CachePluginContext ctx) {
+                if (!ctx.igniteCacheConfiguration().getName().equals(FOR_OTHER_CLUSTER_ID))
+                    return null;
+
+                return new AbstractCachePluginProvider() {
+                    @Override public @Nullable Object createComponent(Class cls) {
+                        if (cls != CacheConflictResolutionManager.class)
+                            return null;
+
+                        return new TestCacheConflictResolutionManager();
+                    }
+                };
+            }
+        });
+
+        return cfg;
+    }
+
+    /** Simplest CDC test with usage of {@link IgniteInternalCache#putAllConflict(Map)}. */
+    @Test
+    public void testReadAllKeysFromOtherCluster() throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-conflict-resolver");
+
+        IgniteEx ign = startGrid(cfg);
+
+        ign.context().cache().context().versions().dataCenterId(DFLT_CLUSTER_ID);
+        ign.cluster().state(ACTIVE);
+
+        UserCdcConsumer cnsmr = new UserCdcConsumer() {
+            @Override public void checkEvent(CdcEvent evt) {
+                assertEquals(DFLT_CLUSTER_ID, evt.version().clusterId());
+                assertEquals(OTHER_CLUSTER_ID, evt.version().otherClusterVersion().clusterId());
+            }
+        };
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+        IgniteCache<Integer, User> cache = ign.getOrCreateCache(FOR_OTHER_CLUSTER_ID);
+
+        addAndWaitForConsumption(cnsmr, cdc, cache, null, this::addConflictData, 0, KEYS_CNT, getTestTimeout());
+    }
+
+    /** */
+    @Test
+    public void testOrderIncrease() throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-0");
+
+        IgniteEx ign = startGrid(cfg);
+
+        ign.cluster().state(ACTIVE);
+
+        AtomicLong updCntr = new AtomicLong(0);
+
+        CdcConsumer cnsmr = new CdcConsumer() {
+            private long order = -1;
+
+            @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+                evts.forEachRemaining(evt -> {
+                    assertEquals(KEY_TO_UPD, evt.key());
+
+                    assertTrue(evt.version().order() > order);
+
+                    order = evt.version().order();
+
+                    updCntr.incrementAndGet();
+                });
+
+                return true;
+            }
+
+            @Override public void start() {
+                // No-op.
+            }
+
+            @Override public void stop() {
+                // No-op.
+            }
+        };
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+        IgniteCache<Integer, User> cache = ign.getOrCreateCache("my-cache");
+
+        IgniteInternalFuture<?> fut = runAsync(cdc);
+
+        // Update the same key several time.
+        // Expect {@link CacheEntryVersion#order()} will monotically increase.
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.put(KEY_TO_UPD, createUser(i));
+
+        assertTrue(waitForCondition(() -> updCntr.get() == KEYS_CNT, getTestTimeout()));
+
+        fut.cancel();
+    }
+
+    /** */
+    private void addConflictData(IgniteCache<Integer, User> cache, int from, int to) {
+        try {
+            IgniteEx ign = (IgniteEx)G.allGrids().get(0);
+
+            IgniteInternalCache<Integer, User> intCache = ign.cachex(cache.getName());
+
+            Map<KeyCacheObject, GridCacheDrInfo> drMap = new HashMap<>();
+
+            for (int i = from; i < to; i++) {
+                KeyCacheObject key = new KeyCacheObjectImpl(i, null, intCache.affinity().partition(i));
+                CacheObject val =
+                    new CacheObjectImpl(createUser(i), null);
+
+                val.prepareMarshal(intCache.context().cacheObjectContext());
+
+                drMap.put(key, new GridCacheDrInfo(val, new GridCacheVersion(1, i, 1, OTHER_CLUSTER_ID)));
+            }
+
+            intCache.putAllConflict(drMap);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** */
+    public static class TestCacheConflictResolutionManager<K, V> extends GridCacheManagerAdapter<K, V>
+        implements CacheConflictResolutionManager<K, V> {
+
+        /** {@inheritDoc} */
+        @Override public CacheVersionConflictResolver conflictResolver() {
+            return new CacheVersionConflictResolver() {
+                @Override public <K1, V1> GridCacheVersionConflictContext<K1, V1> resolve(
+                    CacheObjectValueContext ctx,
+                    GridCacheVersionedEntryEx<K1, V1> oldEntry,
+                    GridCacheVersionedEntryEx<K1, V1> newEntry,
+                    boolean atomicVerComparator
+                ) {
+                    GridCacheVersionConflictContext<K1, V1> res =
+                        new GridCacheVersionConflictContext<>(ctx, oldEntry, newEntry);
+
+                    res.useNew();
+
+                    return res;
+                }
+            };
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java
new file mode 100644
index 0000000..9f1e55f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java
@@ -0,0 +1,414 @@
+/*
+ * 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.cdc;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.util.typedef.F;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE;
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/** */
+@RunWith(Parameterized.class)
+public class CdcSelfTest extends AbstractCdcTest {
+    /** */
+    public static final String TX_CACHE_NAME = "tx-cache";
+
+    /** */
+    public static final int WAL_ARCHIVE_TIMEOUT = 5_000;
+
+    /** */
+    @Parameterized.Parameter
+    public boolean specificConsistentId;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public WALMode walMode;
+
+    /** */
+    @Parameterized.Parameters(name = "specificConsistentId={0}, walMode={1}")
+    public static Collection<?> parameters() {
+        return Arrays.asList(new Object[][] {
+            {true, WALMode.FSYNC},
+            {false, WALMode.FSYNC},
+            {true, WALMode.LOG_ONLY},
+            {false, WALMode.LOG_ONLY},
+            {true, WALMode.BACKGROUND},
+            {false, WALMode.BACKGROUND}
+        });
+    }
+
+    /** Consistent id. */
+    private UUID consistentId = UUID.randomUUID();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (specificConsistentId)
+            cfg.setConsistentId(consistentId);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setCdcEnabled(true)
+            .setWalMode(walMode)
+            .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT)
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true)));
+
+        cfg.setCacheConfiguration(
+            new CacheConfiguration<>(TX_CACHE_NAME).setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+        );
+
+        return cfg;
+    }
+
+    /** Simplest CDC test. */
+    @Test
+    public void testReadAllKeys() throws Exception {
+        // Read all records from iterator.
+        readAll(new UserCdcConsumer());
+
+        // Read one record per call.
+        readAll(new UserCdcConsumer() {
+            @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+                super.onEvents(Collections.singleton(evts.next()).iterator());
+
+                return false;
+            }
+        });
+
+        // Read one record per call and commit.
+        readAll(new UserCdcConsumer() {
+            @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+                super.onEvents(Collections.singleton(evts.next()).iterator());
+
+                return true;
+            }
+        });
+    }
+
+    /** */
+    private void readAll(UserCdcConsumer cnsmr) throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-0");
+
+        Ignite ign = startGrid(cfg);
+
+        ign.cluster().state(ACTIVE);
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+        IgniteCache<Integer, User> cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, User> txCache = ign.getOrCreateCache(TX_CACHE_NAME);
+
+        addAndWaitForConsumption(
+            cnsmr,
+            cdc,
+            cache,
+            txCache,
+            CdcSelfTest::addData,
+            0,
+            KEYS_CNT + 3,
+            getTestTimeout()
+        );
+
+        removeData(cache, 0, KEYS_CNT);
+
+        IgniteInternalFuture<?> rmvFut = runAsync(cdc);
+
+        assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr));
+
+        rmvFut.cancel();
+
+        assertTrue(cnsmr.stopped());
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    @Test
+    public void testReadBeforeGracefulShutdown() throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-0");
+
+        Ignite ign = startGrid(cfg);
+
+        ign.cluster().state(ACTIVE);
+
+        CountDownLatch cnsmrStarted = new CountDownLatch(1);
+        CountDownLatch startProcEvts = new CountDownLatch(1);
+
+        UserCdcConsumer cnsmr = new UserCdcConsumer() {
+            @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+                cnsmrStarted.countDown();
+
+                try {
+                    startProcEvts.await(getTestTimeout(), TimeUnit.MILLISECONDS);
+                }
+                catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+
+                return super.onEvents(evts);
+            }
+        };
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+        runAsync(cdc);
+
+        IgniteCache<Integer, User> cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        addData(cache, 0, KEYS_CNT);
+
+        // Make sure all streamed data will become available for consumption.
+        Thread.sleep(2 * WAL_ARCHIVE_TIMEOUT);
+
+        cnsmrStarted.await(getTestTimeout(), TimeUnit.MILLISECONDS);
+
+        // Initiate graceful shutdown.
+        cdc.stop();
+
+        startProcEvts.countDown();
+
+        assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr));
+        assertTrue(waitForCondition(cnsmr::stopped, getTestTimeout()));
+
+        List<Integer> keys = cnsmr.data(UPDATE, cacheId(DEFAULT_CACHE_NAME));
+
+        assertEquals(KEYS_CNT, keys.size());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            assertTrue(keys.contains(i));
+    }
+
+    /** */
+    @Test
+    public void testMultiNodeConsumption() throws Exception {
+        IgniteEx ign1 = startGrid(0);
+
+        if (specificConsistentId)
+            consistentId = UUID.randomUUID();
+
+        IgniteEx ign2 = startGrid(1);
+
+        ign1.cluster().state(ACTIVE);
+
+        IgniteCache<Integer, User> cache = ign1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        // Adds data concurrently with CDC start.
+        IgniteInternalFuture<?> addDataFut = runAsync(() -> addData(cache, 0, KEYS_CNT));
+
+        UserCdcConsumer cnsmr1 = new UserCdcConsumer();
+        UserCdcConsumer cnsmr2 = new UserCdcConsumer();
+
+        IgniteConfiguration cfg1 = ign1.configuration();
+        IgniteConfiguration cfg2 = ign2.configuration();
+
+        CdcMain cdc1 = new CdcMain(cfg1, null, cdcConfig(cnsmr1));
+        CdcMain cdc2 = new CdcMain(cfg2, null, cdcConfig(cnsmr2));
+
+        IgniteInternalFuture<?> fut1 = runAsync(cdc1);
+        IgniteInternalFuture<?> fut2 = runAsync(cdc2);
+
+        addDataFut.get(getTestTimeout());
+
+        addDataFut = runAsync(() -> addData(cache, KEYS_CNT, KEYS_CNT * 2));
+
+        addDataFut.get(getTestTimeout());
+
+        assertTrue(waitForSize(KEYS_CNT * 2, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr1, cnsmr2));
+
+        assertFalse(cnsmr1.stopped());
+        assertFalse(cnsmr2.stopped());
+
+        fut1.cancel();
+        fut2.cancel();
+
+        assertTrue(cnsmr1.stopped());
+        assertTrue(cnsmr2.stopped());
+
+        removeData(cache, 0, KEYS_CNT * 2);
+
+        IgniteInternalFuture<?> rmvFut1 = runAsync(cdc1);
+        IgniteInternalFuture<?> rmvFut2 = runAsync(cdc2);
+
+        assertTrue(waitForSize(KEYS_CNT * 2, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr1, cnsmr2));
+
+        rmvFut1.cancel();
+        rmvFut2.cancel();
+
+        assertTrue(cnsmr1.stopped());
+        assertTrue(cnsmr2.stopped());
+    }
+
+    /** */
+    @Test
+    public void testCdcSingleton() throws Exception {
+        IgniteEx ign = startGrid(0);
+
+        UserCdcConsumer cnsmr1 = new UserCdcConsumer();
+        UserCdcConsumer cnsmr2 = new UserCdcConsumer();
+
+        IgniteInternalFuture<?> fut1 = runAsync(new CdcMain(ign.configuration(), null, cdcConfig(cnsmr1)));
+        IgniteInternalFuture<?> fut2 = runAsync(new CdcMain(ign.configuration(), null, cdcConfig(cnsmr2)));
+
+        assertTrue(waitForCondition(() -> fut1.isDone() || fut2.isDone(), getTestTimeout()));
+
+        assertEquals(fut1.error() == null, fut2.error() != null);
+
+        if (fut1.isDone()) {
+            fut2.cancel();
+
+            assertTrue(cnsmr2.stopped());
+        }
+        else {
+            fut1.cancel();
+
+            assertTrue(cnsmr1.stopped());
+        }
+    }
+
+    /** */
+    @Test
+    public void testReReadWhenStateWasNotStored() throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-0");
+
+        IgniteEx ign = startGrid(cfg);
+
+        ign.cluster().state(ACTIVE);
+
+        IgniteCache<Integer, User> cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        addData(cache, 0, KEYS_CNT);
+
+        for (int i = 0; i < 3; i++) {
+            UserCdcConsumer cnsmr = new UserCdcConsumer() {
+                @Override protected boolean commit() {
+                    return false;
+                }
+            };
+
+            CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+            IgniteInternalFuture<?> fut = runAsync(cdc);
+
+            assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr));
+
+            fut.cancel();
+
+            assertTrue(cnsmr.stopped());
+        }
+
+        AtomicBoolean consumeHalf = new AtomicBoolean(true);
+        AtomicBoolean halfCommitted = new AtomicBoolean(false);
+
+        int half = KEYS_CNT / 2;
+
+        UserCdcConsumer cnsmr = new UserCdcConsumer() {
+            @Override public boolean onEvents(Iterator<CdcEvent> evts) {
+                if (consumeHalf.get() && F.size(data(UPDATE, cacheId(DEFAULT_CACHE_NAME))) == half) {
+                    // This means that state committed as a result of the previous call.
+                    halfCommitted.set(true);
+
+                    return false;
+                }
+
+                while (evts.hasNext()) {
+                    CdcEvent evt = evts.next();
+
+                    if (!evt.primary())
+                        continue;
+
+                    data.computeIfAbsent(
+                        F.t(evt.value() == null ? DELETE : UPDATE, evt.cacheId()),
+                        k -> new ArrayList<>()).add((Integer)evt.key()
+                    );
+
+                    if (consumeHalf.get())
+                        return F.size(data(UPDATE, cacheId(DEFAULT_CACHE_NAME))) == half;
+                }
+
+                return true;
+            }
+        };
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr));
+
+        IgniteInternalFuture<?> fut = runAsync(cdc);
+
+        waitForSize(half, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr);
+
+        waitForCondition(halfCommitted::get, getTestTimeout());
+
+        fut.cancel();
+
+        assertTrue(cnsmr.stopped());
+
+        removeData(cache, 0, KEYS_CNT);
+
+        consumeHalf.set(false);
+
+        fut = runAsync(cdc);
+
+        waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr);
+        waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr);
+
+        fut.cancel();
+
+        assertTrue(cnsmr.stopped());
+    }
+
+    /** */
+    public static void addData(IgniteCache<Integer, User> cache, int from, int to) {
+        for (int i = from; i < to; i++)
+            cache.put(i, createUser(i));
+    }
+
+    /** */
+    private void removeData(IgniteCache<Integer, ?> cache, int from, int to) {
+        for (int i = from; i < to; i++)
+            cache.remove(i);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index c406054..4d86ed3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -84,7 +84,7 @@
 
     /** {@inheritDoc} */
     @Override public boolean initialValue(CacheObject val, GridCacheVersion ver, long ttl, long expireTime,
-        boolean preload, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore) {
+        boolean preload, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore, boolean primary) {
         assert false;
 
         return false;
@@ -711,6 +711,7 @@
         AffinityTopologyVersion topVer,
         GridDrType drType,
         boolean fromStore,
+        boolean primary,
         CacheDataRow row
     ) throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert false;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java
index 03548c2..de8734e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java
@@ -259,6 +259,7 @@
                 false,
                 AffinityTopologyVersion.NONE,
                 GridDrType.DR_NONE,
+                false,
                 false);
 
             assertTrue("iterableKey " + key + " already inited", init);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
index d82d263..55b6b61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java
@@ -158,7 +158,7 @@
 
         params.bufferSize(1024 * 1024);
         params.filesOrDirs(walDir, walArchiveDir);
-        params.filter((type, pointer) -> type == WALRecord.RecordType.DATA_RECORD);
+        params.filter((type, pointer) -> type == WALRecord.RecordType.DATA_RECORD_V2);
 
         int cacheId = CU.cacheId(TX_CACHE_NAME);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
index 3fa1611..0caec57 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
@@ -358,7 +358,7 @@
                     cctx.affinity().partition(i), i, new MvccVersionImpl(1000L, 10L, i + 1 /* Non-zero */)) :
                 new DataEntry(cctx.cacheId(), key, val, op, null, cctx.cache().nextVersion(),
                     0L,
-                    cctx.affinity().partition(i), i));
+                    cctx.affinity().partition(i), i, DataEntry.EMPTY_FLAGS));
         }
 
         UUID cpId = UUID.randomUUID();
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
index 96a32f8..9240892 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
@@ -31,7 +31,7 @@
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -40,7 +40,7 @@
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.parseSubFolderName;
 
 /**
  * Test for new and old style persistent storage folders generation
@@ -365,7 +365,7 @@
                 " but actual class is " + (consistentId == null ? null : consistentId.getClass()),
             consistentId instanceof UUID);
 
-        return PdsConsistentIdProcessor.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
+        return PdsFolderResolver.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
     }
 
     /**
@@ -655,12 +655,12 @@
      * @throws IgniteCheckedException if failed.
      */
     @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
-        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsFolderResolver.DB_DEFAULT_FOLDER);
         final Set<Integer> indexes = new TreeSet<>();
-        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+        final File[] files = curFolder.listFiles(PdsFolderResolver.DB_SUBFOLDERS_NEW_STYLE_FILTER);
 
         for (File file : files) {
-            final PdsConsistentIdProcessor.FolderCandidate uid = parseSubFolderName(file, log);
+            final PdsFolderResolver.FolderCandidate uid = parseSubFolderName(file, log);
 
             if (uid != null)
                 indexes.add(uid.nodeIndex());
@@ -679,7 +679,7 @@
         assertDirectoryExist(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH, subDirName);
         assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_PATH, subDirName);
         assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
-        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+        assertDirectoryExist(PdsFolderResolver.DB_DEFAULT_FOLDER, subDirName);
     }
 
     /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java
index d45cf90..cbb3df7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java
@@ -782,7 +782,8 @@
                     new GridCacheVersion(0, 1, 1, 0),
                     0,
                     0,
-                    0
+                    0,
+                    DataEntry.EMPTY_FLAGS
                 )));
 
                 File walDir = U.field(walMgr, "walWorkDir");
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index efa2924..afd756d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -45,6 +45,7 @@
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteEvents;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
@@ -79,6 +80,7 @@
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.testframework.MvccFeatureChecker;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.NotNull;
@@ -88,14 +90,15 @@
 import org.junit.Test;
 
 import static java.util.Arrays.fill;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_COMPACTED;
-import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.MVCC_DATA_RECORD;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName;
 
 /**
  * Test suite for WAL segments reader and event generator.
@@ -119,6 +122,9 @@
      */
     private int archiveIncompleteSegmentAfterInactivityMs;
 
+    /** Force archive timeout in milliseconds. */
+    private int forceArchiveSegmentMs;
+
     /** Custom wal mode. */
     private WALMode customWalMode;
 
@@ -128,6 +134,15 @@
     /** Whether to enable WAL archive compaction. */
     private boolean enableWalCompaction;
 
+    /** Backup count. */
+    private int backupCnt;
+
+    /** DataEntry from primary flag. */
+    private boolean primary = true;
+
+    /** DataEntry during rebalacne flag. */
+    private boolean rebalance;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -138,6 +153,7 @@
         ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
         ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
         ccfg.setIndexedTypes(Integer.class, IndexedObject.class);
+        ccfg.setBackups(backupCnt);
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -156,6 +172,9 @@
         if (archiveIncompleteSegmentAfterInactivityMs > 0)
             dsCfg.setWalAutoArchiveAfterInactivity(archiveIncompleteSegmentAfterInactivityMs);
 
+        if (forceArchiveSegmentMs > 0)
+            dsCfg.setWalForceArchiveTimeout(forceArchiveSegmentMs);
+
         String workDir = U.defaultWorkDirectory();
         File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false);
         File wal = new File(db, "wal");
@@ -257,13 +276,18 @@
 
                 WALRecord walRecord = tup.get2();
 
-                if (walRecord.type() == DATA_RECORD || walRecord.type() == MVCC_DATA_RECORD) {
+                if (walRecord.type() == DATA_RECORD_V2 || walRecord.type() == MVCC_DATA_RECORD) {
                     DataRecord record = (DataRecord)walRecord;
 
                     for (DataEntry entry : record.writeEntries()) {
                         KeyCacheObject key = entry.key();
                         CacheObject val = entry.value();
 
+                        if (walRecord.type() == DATA_RECORD_V2) {
+                            assertEquals(primary, (entry.flags() & DataEntry.PRIMARY_FLAG) != 0);
+                            assertEquals(rebalance, (entry.flags() & DataEntry.PRELOAD_FLAG) != 0);
+                        }
+
                         if (DUMP_RECORDS)
                             log.info("Op: " + entry.op() + ", Key: " + key + ", Value: " + val);
                     }
@@ -341,6 +365,45 @@
     }
 
     /**
+     * Tests force time out based WAL segment archiving.
+     *
+     * @throws Exception if failure occurs.
+     */
+    @Test
+    public void testForceArchiveSegment() throws Exception {
+        AtomicBoolean waitingForEvt = new AtomicBoolean();
+
+        CountDownLatch forceArchiveSegment = new CountDownLatch(1);
+
+        forceArchiveSegmentMs = 1000;
+
+        Ignite ignite = startGrid();
+
+        ignite.cluster().state(ACTIVE);
+
+        IgniteEvents evts = ignite.events();
+
+        evts.localListen(e -> {
+            if (waitingForEvt.get())
+                forceArchiveSegment.countDown();
+
+            return true;
+        }, EVT_WAL_SEGMENT_ARCHIVED);
+
+        putDummyRecords(ignite, 100);
+
+        waitingForEvt.set(true); // Flag for skipping regular log() and rollOver().
+
+        putDummyRecords(ignite, 1);
+
+        boolean recordedAfterSleep = forceArchiveSegment.await(forceArchiveSegmentMs + getTestTimeout(), TimeUnit.MILLISECONDS);
+
+        stopGrid();
+
+        assertTrue(recordedAfterSleep);
+    }
+
+    /**
      * Tests time out based WAL segment archiving.
      *
      * @throws Exception if failure occurs.
@@ -1002,6 +1065,189 @@
      * @throws Exception if failed.
      */
     @Test
+    public void testPrimaryFlagOnTwoNodes() throws Exception {
+        backupCnt = 1;
+
+        IgniteEx ignite = startGrid("node0");
+        Ignite ignite1 = startGrid(1);
+
+        ignite.cluster().state(ACTIVE);
+
+        IgniteCache<Integer, IndexedObject> cache = ignite.cache(CACHE_NAME);
+
+        backupCnt = 0;
+
+        int cntEntries = 100;
+
+        List<Integer> keys = findKeys(ignite.localNode(), cache, cntEntries, 0, 0);
+
+        Map<Integer, IndexedObject> map = new TreeMap<>();
+
+        for (Integer key : keys)
+            map.putIfAbsent(key, new IndexedObject(key));
+
+        cache.putAll(map);
+
+        ignite.cluster().active(false);
+
+        String subfolderName1 = genDbSubfolderName(ignite, 0);
+        String subfolderName2 = genDbSubfolderName(ignite1, 1);
+
+        stopAllGrids();
+
+        String workDir = U.defaultWorkDirectory();
+
+        IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
+
+        Map<GridCacheOperation, Integer> operationsFound = new EnumMap<>(GridCacheOperation.class);
+
+        IgniteInClosure<DataRecord> drHnd = dataRecord -> {
+            List<? extends DataEntry> entries = dataRecord.writeEntries();
+
+            for (DataEntry entry : entries) {
+                GridCacheOperation op = entry.op();
+                Integer cnt = operationsFound.get(op);
+
+                operationsFound.put(op, cnt == null ? 1 : (cnt + 1));
+            }
+        };
+
+        scanIterateAndCount(
+            factory,
+            createIteratorParametersBuilder(workDir, subfolderName1)
+                .filesOrDirs(
+                    workDir + "/db/wal/" + subfolderName1,
+                    workDir + "/db/wal/archive/" + subfolderName1
+                ),
+            1,
+            1,
+            null, drHnd
+        );
+
+        primary = false;
+
+        scanIterateAndCount(
+            factory,
+            createIteratorParametersBuilder(workDir, subfolderName2)
+                .filesOrDirs(
+                    workDir + "/db/wal/" + subfolderName2,
+                    workDir + "/db/wal/archive/" + subfolderName2
+                ),
+            1,
+            1,
+            null,
+            drHnd
+        );
+    }
+
+    /**
+     * Tests transaction generation and WAL for putAll cache operation.
+     *
+     * @throws Exception if failed.
+     */
+    @Test
+    @WithSystemProperty(key = IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING, value = "false")
+    public void testRebalanceFlag() throws Exception {
+        backupCnt = 1;
+
+        IgniteEx ignite = startGrid("node0");
+        Ignite ignite1 = startGrid(1);
+
+        ignite.cluster().state(ACTIVE);
+
+        IgniteCache<Integer, IndexedObject> cache = ignite.cache(CACHE_NAME);
+
+        int cntEntries = 100;
+
+        List<Integer> keys = findKeys(ignite.localNode(), cache, cntEntries, 0, 0);
+
+        Map<Integer, IndexedObject> map = new TreeMap<>();
+
+        for (Integer key : keys)
+            map.putIfAbsent(key, new IndexedObject(key));
+
+        cache.putAll(map);
+
+        Ignite ignite2 = startGrid(2);
+
+        ignite.cluster().setBaselineTopology(ignite2.cluster().topologyVersion());
+
+        backupCnt = 0;
+
+        awaitPartitionMapExchange(false, true, null);
+
+        String subfolderName1 = genDbSubfolderName(ignite, 0);
+        String subfolderName2 = genDbSubfolderName(ignite1, 1);
+        String subfolderName3 = genDbSubfolderName(ignite2, 2);
+
+        stopAllGrids();
+
+        String workDir = U.defaultWorkDirectory();
+
+        IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log);
+
+        Map<GridCacheOperation, Integer> operationsFound = new EnumMap<>(GridCacheOperation.class);
+
+        IgniteInClosure<DataRecord> drHnd = dataRecord -> {
+            List<? extends DataEntry> entries = dataRecord.writeEntries();
+
+            for (DataEntry entry : entries) {
+                GridCacheOperation op = entry.op();
+                Integer cnt = operationsFound.get(op);
+
+                operationsFound.put(op, cnt == null ? 1 : (cnt + 1));
+            }
+        };
+
+        scanIterateAndCount(
+            factory,
+            createIteratorParametersBuilder(workDir, subfolderName1)
+                .filesOrDirs(
+                    workDir + "/db/wal/" + subfolderName1,
+                    workDir + "/db/wal/archive/" + subfolderName1
+                ),
+            1,
+            1,
+            null, drHnd
+        );
+
+        primary = false;
+
+        scanIterateAndCount(
+            factory,
+            createIteratorParametersBuilder(workDir, subfolderName2)
+                .filesOrDirs(
+                    workDir + "/db/wal/" + subfolderName2,
+                    workDir + "/db/wal/archive/" + subfolderName2
+                ),
+            1,
+            1,
+            null,
+            drHnd
+        );
+
+        rebalance = true;
+
+        scanIterateAndCount(
+            factory,
+            createIteratorParametersBuilder(workDir, subfolderName3)
+                .filesOrDirs(
+                    workDir + "/db/wal/" + subfolderName3,
+                    workDir + "/db/wal/archive/" + subfolderName3
+                ),
+            1,
+            0,
+            null,
+            drHnd
+        );
+    }
+
+    /**
+     * Tests transaction generation and WAL for putAll cache operation.
+     *
+     * @throws Exception if failed.
+     */
+    @Test
     public void testPutAllTxIntoTwoNodes() throws Exception {
         Ignite ignite = startGrid("node0");
         Ignite ignite1 = startGrid(1);
@@ -1351,7 +1597,7 @@
 
                 //noinspection EnumSwitchStatementWhichMissesCases
                 switch (type) {
-                    case DATA_RECORD:
+                    case DATA_RECORD_V2:
                         // Fallthrough.
                     case MVCC_DATA_RECORD: {
                         assert walRecord instanceof DataRecord;
@@ -1364,6 +1610,11 @@
                         List<DataEntry> entries = dataRecord.writeEntries();
 
                         for (DataEntry entry : entries) {
+                            if (walRecord.type() == DATA_RECORD_V2) {
+                                assertEquals(primary, (entry.flags() & DataEntry.PRIMARY_FLAG) != 0);
+                                assertEquals(rebalance, (entry.flags() & DataEntry.PRELOAD_FLAG) != 0);
+                            }
+
                             GridCacheVersion globalTxId = entry.nearXidVersion();
 
                             Object unwrappedKeyObj;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
index b9507d1..ae08924 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java
@@ -409,6 +409,7 @@
                         topVer,
                         DR_NONE,
                         false,
+                        false,
                         null);
 
                     assertTrue(success);
diff --git a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java
index 4687ca9..fa9f536 100644
--- a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java
@@ -20,7 +20,8 @@
 import java.util.UUID;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
 import org.junit.Test;
 
@@ -43,7 +44,7 @@
         log = new JavaLogger();
 
         ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory());
-        ((LoggerNodeIdAware)log).setNodeId(UUID.fromString("00000000-1111-2222-3333-444444444444"));
+        ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.fromString("00000000-1111-2222-3333-444444444444"));
 
         System.out.println(log.toString());
 
@@ -67,5 +68,20 @@
 
         // Ensure we don't get pattern, only actual file name is allowed here.
         assert !log.fileName().contains("%");
+        assert log.fileName().contains("ignite");
+
+        System.clearProperty("java.util.logging.config.file");
+        GridTestUtils.setFieldValue(JavaLogger.class, JavaLogger.class, "inited", false);
+
+        log = new JavaLogger();
+
+        ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory());
+        ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode("other-app", UUID.fromString("00000000-1111-2222-3333-444444444444"));
+
+        assert log.fileName() != null;
+
+        // Ensure we don't get pattern, only actual file name is allowed here.
+        assert !log.fileName().contains("%");
+        assert log.fileName().contains("other-app");
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java b/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java
new file mode 100644
index 0000000..c301e22
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java
@@ -0,0 +1,70 @@
+/*
+ * 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.plugin;
+
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * No-op test implementation of {@link CachePluginProvider} which allows to avoid redundant boilerplate code.
+ */
+public abstract class AbstractCachePluginProvider implements CachePluginProvider {
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIgniteStart() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIgniteStop(boolean cancel) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void validate() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void validateRemote(CacheConfiguration locCfg, CacheConfiguration rmtCfg,
+        ClusterNode rmtNode) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object unwrapCacheEntry(Cache.Entry entry, Class cls) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object createComponent(Class cls) {
+        return null;
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java
index fc9f38f..98841d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java
@@ -23,14 +23,15 @@
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.apache.log4j.Layout;
 import org.apache.log4j.RollingFileAppender;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Log4J {@link org.apache.log4j.RollingFileAppender} with added support for grid node IDs.
  */
-public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAware {
+public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAndApplicationAware {
     /** Node ID. */
     private UUID nodeId;
 
@@ -79,8 +80,7 @@
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-    @Override public synchronized void setNodeId(UUID nodeId) {
+    @Override public synchronized void setApplicationAndNode(@Nullable String application, UUID nodeId) {
         A.notNull(nodeId, "nodeId");
 
         this.nodeId = nodeId;
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
index 4464828..6563e1c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
@@ -33,7 +33,7 @@
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Category;
 import org.apache.log4j.ConsoleAppender;
@@ -77,7 +77,7 @@
  * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger
  * injection.
  */
-public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware {
+public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware {
     /** Appenders. */
     private static Collection<FileAppender> fileAppenders = new GridConcurrentHashSet<>();
 
@@ -406,14 +406,14 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void setNodeId(UUID nodeId) {
+    @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) {
         A.notNull(nodeId, "nodeId");
 
         this.nodeId = nodeId;
 
         for (FileAppender a : fileAppenders) {
-            if (a instanceof LoggerNodeIdAware) {
-                ((LoggerNodeIdAware)a).setNodeId(nodeId);
+            if (a instanceof LoggerNodeIdAndApplicationAware) {
+                ((LoggerNodeIdAndApplicationAware)a).setApplicationAndNode(application, nodeId);
 
                 a.activateOptions();
             }
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java
index 8fc3076..f7aefe8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java
@@ -112,8 +112,10 @@
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_UPDATE_RECORD;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2;
+import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V3;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD_V2;
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.EXCHANGE;
@@ -171,6 +173,7 @@
             put(TX_RECORD, RecordUtils::buildTxRecord);
             put(PAGE_RECORD, RecordUtils::buildPageSnapshot);
             put(DATA_RECORD, RecordUtils::buildDataRecord);
+            put(DATA_RECORD_V2, RecordUtils::buildDataRecord);
             put(CHECKPOINT_RECORD, RecordUtils::buildCheckpointRecord);
             put(HEADER_RECORD, RecordUtils::buildHeaderRecord);
             put(INIT_NEW_PAGE_RECORD, RecordUtils::buildInitNewPageRecord);
@@ -232,6 +235,7 @@
             put(ENCRYPTED_DATA_RECORD, RecordUtils::buildEncryptedDataRecord);
             put(ENCRYPTED_RECORD_V2, RecordUtils::buildEncryptedRecordV2);
             put(ENCRYPTED_DATA_RECORD_V2, RecordUtils::buildEncryptedDataRecordV2);
+            put(ENCRYPTED_DATA_RECORD_V3, RecordUtils::buildEncryptedDataRecordV3);
             put(MVCC_DATA_RECORD, RecordUtils::buildMvccDataRecord);
             put(MVCC_TX_RECORD, RecordUtils::buildMvccTxRecord);
             put(CONSISTENT_CUT, RecordUtils::buildConsistentCutRecord);
@@ -595,6 +599,11 @@
     }
 
     /** **/
+    public static UnsupportedWalRecord buildEncryptedDataRecordV3() {
+        return new UnsupportedWalRecord(ENCRYPTED_DATA_RECORD_V3);
+    }
+
+    /** **/
     public static MvccDataRecord buildMvccDataRecord() {
         return new MvccDataRecord(Collections.emptyList(), 1);
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index e7ac51f..f2e1cff 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -20,6 +20,8 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import org.apache.ignite.cdc.CdcCacheVersionTest;
+import org.apache.ignite.cdc.CdcSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceHeuristicsTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceTwoPartsInDifferentCheckpointsTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest;
@@ -209,6 +211,10 @@
         GridTestUtils.addTestIfNeeded(suite, IgnitePdsReserveWalSegmentsWithCompactionTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteWalReplayingAfterRestartTest.class, ignoredTests);
 
+        // CDC tests.
+        GridTestUtils.addTestIfNeeded(suite, CdcSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CdcCacheVersionTest.class, ignoredTests);
+
         // new style folders with generated consistent ID test
         GridTestUtils.addTestIfNeeded(suite, IgniteUidAsConsistentIdMigrationTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteWalSerializerVersionTest.class, ignoredTests);
diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
index 49edc6c..fd660de 100644
--- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
+++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
@@ -65,7 +65,8 @@
             dataEntry.writeVersion(),
             dataEntry.expireTime(),
             dataEntry.partitionId(),
-            dataEntry.partitionCounter()
+            dataEntry.partitionCounter(),
+            dataEntry.flags()
         );
 
         this.source = dataEntry;
diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java
index 993ae88..84be225 100644
--- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java
+++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java
@@ -127,7 +127,9 @@
 
         if (type == WALRecord.RecordType.PAGE_RECORD)
             registerPageSnapshot((PageSnapshot)record);
-        else if (type == WALRecord.RecordType.DATA_RECORD || type == WALRecord.RecordType.MVCC_DATA_RECORD)
+        else if (type == WALRecord.RecordType.DATA_RECORD
+            || type == WALRecord.RecordType.DATA_RECORD_V2
+            || type == WALRecord.RecordType.MVCC_DATA_RECORD)
             registerDataRecord((DataRecord)record);
         else if (type == WALRecord.RecordType.TX_RECORD || type == WALRecord.RecordType.MVCC_TX_RECORD)
             registerTxRecord((TxRecord)record);
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
index 52c42c7..ef4f799 100644
--- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java
@@ -393,7 +393,7 @@
             "binaryMetadataFileStoreDir=" + binaryMetadataDir.getAbsolutePath(),
             "marshallerMappingFileStoreDir=" + marshallerDir.getAbsolutePath(),
             "keepBinary=false",
-            "recordTypes=DATA_RECORD,TX_RECORD",
+            "recordTypes=DATA_RECORD_V2,TX_RECORD",
             "walTimeFromMillis=1575158400000",
             "walTimeToMillis=1577836740999",
             "recordContainsText=search string",
@@ -408,7 +408,7 @@
         Assert.assertEquals(binaryMetadataDir, parseArgs.getBinaryMetadataFileStoreDir());
         Assert.assertEquals(marshallerDir, parseArgs.getMarshallerMappingFileStoreDir());
         Assert.assertFalse(parseArgs.isKeepBinary());
-        Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.DATA_RECORD));
+        Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.DATA_RECORD_V2));
         Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.TX_RECORD));
         Assert.assertEquals(1575158400000L, (long)parseArgs.getFromTime());
         Assert.assertEquals(1577836740999L, (long)parseArgs.getToTime());
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
index bb4cab9..00133ce 100644
--- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java
@@ -298,7 +298,8 @@
             new GridCacheVersion(),
             0,
             0,
-            0
+            0,
+            DataEntry.EMPTY_FLAGS
         );
 
         byte[] sensitiveDataBytes = SENSITIVE_DATA_VALUE_PREFIX.getBytes(StandardCharsets.UTF_8);
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java
index 8edd16b..27ae527 100644
--- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java
@@ -319,7 +319,7 @@
 
                     final int len = Integer.reverseBytes(raf.readInt());
 
-                    if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD.index()) {
+                    if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD_V2.index()) {
                         int i = 0;
 
                         int b;
@@ -430,7 +430,7 @@
                 if (recordTypeIndex > 0) {
                     recordTypeIndex--;
 
-                    if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD.index()) {
+                    if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD_V2.index()) {
                         find++;
 
                         if (find == 2) {
diff --git a/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java b/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java
index 72cbc23..877903a 100644
--- a/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java
+++ b/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java
@@ -17,54 +17,11 @@
 
 package org.apache.ignite.platform.plugin.cache;
 
-import javax.cache.Cache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.plugin.CachePluginProvider;
-import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.plugin.AbstractCachePluginProvider;
 
 /**
  * Test cache plugin provider.
  */
-public class PlatformTestCachePluginProvider implements CachePluginProvider {
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIgniteStart() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIgniteStop(boolean cancel) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void validate() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    @Override public void validateRemote(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode)
-        throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Object unwrapCacheEntry(Cache.Entry entry, Class cls) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Object createComponent(Class cls) {
-        return null;
-    }
+public class PlatformTestCachePluginProvider extends AbstractCachePluginProvider {
+    // No-op.
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java
new file mode 100644
index 0000000..1e24d3b
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.cdc;
+
+import java.util.List;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cdc.AbstractCdcTest;
+import org.apache.ignite.cdc.CdcConfiguration;
+import org.apache.ignite.cdc.CdcEvent;
+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.IgniteInternalFuture;
+import org.junit.Test;
+
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE;
+import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE;
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/** */
+public class SqlCdcTest extends AbstractCdcTest {
+    /** */
+    private static final String SARAH = "Sarah Connor";
+
+    /** */
+    public static final String USER = "user";
+
+    /** */
+    public static final String CITY = "city";
+
+    /** */
+    public static final String SPB = "Saint-Petersburg";
+
+    /** */
+    public static final String MSK = "Moscow";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setCdcEnabled(true)
+            .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT)
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true)));
+
+        return cfg;
+    }
+
+    /** Simplest CDC test. */
+    @Test
+    public void testReadAllSQLRows() throws Exception {
+        IgniteConfiguration cfg = getConfiguration("ignite-0");
+
+        IgniteEx ign = startGrid(cfg);
+
+        ign.cluster().state(ACTIVE);
+
+        BinaryCdcConsumer cnsmr = new BinaryCdcConsumer();
+
+        CdcConfiguration cdcCfg = new CdcConfiguration();
+
+        cdcCfg.setConsumer(cnsmr);
+
+        CdcMain cdc = new CdcMain(cfg, null, cdcCfg);
+
+        IgniteInternalFuture<?> fut = runAsync(cdc);
+
+        executeSql(
+            ign,
+            "CREATE TABLE USER(id int, city_id int, name varchar, PRIMARY KEY (id, city_id)) WITH \"CACHE_NAME=user\""
+        );
+
+        executeSql(
+            ign,
+            "CREATE TABLE CITY(id int, name varchar, zip_code varchar(6), PRIMARY KEY (id)) WITH \"CACHE_NAME=city\""
+        );
+
+        for (int i = 0; i < KEYS_CNT; i++) {
+            executeSql(
+                ign,
+                "INSERT INTO USER VALUES(?, ?, ?)",
+                i,
+                42 * i,
+                (i % 2 == 0 ? JOHN : SARAH) + i);
+
+            executeSql(
+                ign,
+                "INSERT INTO CITY VALUES(?, ?, ?)",
+                i,
+                (i % 2 == 0 ? MSK : SPB) + i,
+                Integer.toString(127000 + i));
+        }
+
+        assertTrue(waitForSize(KEYS_CNT, USER, UPDATE, getTestTimeout(), cnsmr));
+        assertTrue(waitForSize(KEYS_CNT, CITY, UPDATE, getTestTimeout(), cnsmr));
+
+        fut.cancel();
+
+        assertEquals(KEYS_CNT, cnsmr.data(UPDATE, cacheId(USER)).size());
+        assertEquals(KEYS_CNT, cnsmr.data(UPDATE, cacheId(CITY)).size());
+
+        assertTrue(cnsmr.stopped());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            executeSql(ign, "DELETE FROM USER WHERE id = ?", i);
+
+        IgniteInternalFuture<?> rmvFut = runAsync(cdc);
+
+        assertTrue(waitForSize(KEYS_CNT, USER, DELETE, getTestTimeout(), cnsmr));
+
+        rmvFut.cancel();
+
+        assertTrue(cnsmr.stopped());
+    }
+
+    /** */
+    public static class BinaryCdcConsumer extends TestCdcConsumer<CdcEvent> {
+        /** {@inheritDoc} */
+        @Override public void checkEvent(CdcEvent evt) {
+            if (evt.value() == null)
+                return;
+
+            if (evt.cacheId() == cacheId(USER)) {
+                int id = ((BinaryObject)evt.key()).field("ID");
+                int cityId = ((BinaryObject)evt.key()).field("CITY_ID");
+
+                assertEquals(42 * id, cityId);
+
+                String name = ((BinaryObject)evt.value()).field("NAME");
+
+                if (id % 2 == 0)
+                    assertTrue(name.startsWith(JOHN));
+                else
+                    assertTrue(name.startsWith(SARAH));
+            }
+            else {
+                int id = (Integer)evt.key();
+                String name = ((BinaryObject)evt.value()).field("NAME");
+                String zipCode = ((BinaryObject)evt.value()).field("ZIP_CODE");
+
+                assertEquals(Integer.toString(127000 + id), zipCode);
+
+                if (id % 2 == 0)
+                    assertTrue(name.startsWith(MSK));
+                else
+                    assertTrue(name.startsWith(SPB));
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public CdcEvent extract(CdcEvent evt) {
+            return evt;
+        }
+    }
+
+    /** */
+    private List<List<?>> executeSql(IgniteEx node, String sqlText, Object... args) {
+        return node.context().query().querySqlFields(new SqlFieldsQuery(sqlText).setArgs(args), true).getAll();
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index b0ae418..9373839 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -92,7 +92,6 @@
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointMarkersStorage;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CompactablePageIO;
@@ -137,6 +136,7 @@
 import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName;
 
 /**
  *
@@ -784,8 +784,7 @@
      * @throws IgniteCheckedException If fail.
      */
     private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException {
-        final String subfolderName
-            = PdsConsistentIdProcessor.genNewStyleSubfolderName(0, UUID.fromString(consId));
+        final String subfolderName = genNewStyleSubfolderName(0, UUID.fromString(consId));
 
         final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 8c5eae1..fbcd303 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyTypeRegistryTest;
+import org.apache.ignite.internal.cdc.SqlCdcTest;
 import org.apache.ignite.internal.metric.SystemViewSelfTest;
 import org.apache.ignite.internal.processors.cache.AffinityKeyNameAndValueFieldNameConflictTest;
 import org.apache.ignite.internal.processors.cache.BigEntryQueryTest;
@@ -666,7 +667,10 @@
 
     InlineIndexKeyTypeRegistryTest.class,
 
-    IgniteStatisticsTestSuite.class
+    IgniteStatisticsTestSuite.class,
+
+    // CDC tests.
+    SqlCdcTest.class
 })
 public class IgniteBinaryCacheQueryTestSuite {
 }
diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
index ec0a5b3..682919b 100644
--- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
+++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
@@ -33,7 +33,7 @@
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Category;
 import org.apache.log4j.ConsoleAppender;
@@ -79,7 +79,7 @@
  * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger
  * injection.
  */
-public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAware {
+public class Log4JLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware, Log4jFileAware {
     /** Appenders. */
     private static Collection<FileAppender> fileAppenders = new GridConcurrentHashSet<>();
 
@@ -500,12 +500,12 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void setNodeId(UUID nodeId) {
+    @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) {
         A.notNull(nodeId, "nodeId");
 
         this.nodeId = nodeId;
 
-        updateFilePath(new Log4jNodeIdFilePath(nodeId));
+        updateFilePath(new Log4jNodeIdFilePath(application, nodeId));
     }
 
     /** {@inheritDoc} */
diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java
index d4d1892..715684d 100644
--- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java
+++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java
@@ -23,6 +23,7 @@
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Closure that generates file path adding node id to filename as a suffix.
@@ -31,22 +32,29 @@
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Applictaion name. */
+    private final @Nullable String app;
+
     /** Node id. */
     private final UUID nodeId;
 
     /**
      * Creates new instance.
      *
+     * @param app Application name.
      * @param id Node id.
      */
-    Log4jNodeIdFilePath(UUID id) {
+    Log4jNodeIdFilePath(@Nullable String app, UUID id) {
+        this.app = app;
         nodeId = id;
     }
 
     /** {@inheritDoc} */
     @Override public String apply(String oldPath) {
+        String fileName = (app != null ? app : "ignite") + ".log";
+
         if (!F.isEmpty(U.IGNITE_LOG_DIR))
-            return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath());
+            return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, fileName).getAbsolutePath());
 
         if (oldPath != null) // fileName could be null if IGNITE_HOME is not defined.
             return U.nodeIdLogFileName(nodeId, oldPath);
@@ -54,7 +62,7 @@
         String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir");
 
         if (tmpDir != null)
-            return U.nodeIdLogFileName(nodeId, new File(tmpDir, "ignite.log").getAbsolutePath());
+            return U.nodeIdLogFileName(nodeId, new File(tmpDir, fileName).getAbsolutePath());
 
         System.err.println("Failed to get tmp directory for log file.");
 
diff --git a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
index 0f5c313..66050c2 100644
--- a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
+++ b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java
@@ -31,7 +31,7 @@
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Marker;
@@ -81,11 +81,14 @@
  * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger
  * injection.
  */
-public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware {
+public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAndApplicationAware {
     /** */
     private static final String NODE_ID = "nodeId";
 
     /** */
+    private static final String APP_ID = "appId";
+
+    /** */
     private static final String CONSOLE_APPENDER = "autoConfiguredIgniteConsoleAppender";
 
     /** */
@@ -384,13 +387,14 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void setNodeId(UUID nodeId) {
+    @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) {
         A.notNull(nodeId, "nodeId");
 
         this.nodeId = nodeId;
 
         // Set nodeId as system variable to be used at configuration.
         System.setProperty(NODE_ID, U.id8(nodeId));
+        System.setProperty(APP_ID, application != null ? application : "ignite");
 
         if (inited) {
             final LoggerContext ctx = impl.getContext();
diff --git a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java
index 3ebd15d..8d0b6fc 100644
--- a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java
+++ b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java
@@ -26,7 +26,7 @@
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.logger.LoggerNodeIdAware;
+import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -70,7 +70,7 @@
         assertTrue(log.toString().contains("Log4J2Logger"));
         assertTrue(log.toString().contains(xml.getPath()));
 
-        ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID());
+        ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID());
 
         checkLog(log);
     }
@@ -93,7 +93,7 @@
         assertTrue(log.toString().contains("Log4J2Logger"));
         assertTrue(log.toString().contains(url.getPath()));
 
-        ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID());
+        ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID());
 
         checkLog(log);
     }
@@ -110,7 +110,7 @@
         assertTrue(log.toString().contains("Log4J2Logger"));
         assertTrue(log.toString().contains(LOG_PATH_TEST));
 
-        ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID());
+        ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID());
 
         checkLog(log);
     }
@@ -137,9 +137,15 @@
     public void testSystemNodeId() throws Exception {
         UUID id = UUID.randomUUID();
 
-        new Log4J2Logger(LOG_PATH_TEST).setNodeId(id);
+        new Log4J2Logger(LOG_PATH_TEST).setApplicationAndNode(null, id);
 
         assertEquals(U.id8(id), System.getProperty("nodeId"));
+        assertEquals("ignite", System.getProperty("appId"));
+
+        new Log4J2Logger(LOG_PATH_TEST).setApplicationAndNode("other-app", id);
+
+        assertEquals(U.id8(id), System.getProperty("nodeId"));
+        assertEquals("other-app", System.getProperty("appId"));
     }
 
     /**
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
index f294afa..41c5cfb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
@@ -168,6 +168,7 @@
                               walArchivePath="abc" walFlushFrequency="00:00:12" walFsyncDelayNanos="13" walHistorySize="14"
                               walMode="Background" walRecordIteratorBufferSize="15" walSegments="16" walSegmentSize="17"
                               walPath="wal-store" writeThrottlingEnabled="true" walAutoArchiveAfterInactivity="00:00:18"
+                              walForceArchiveTimeout="00:00:19"
                               walPageCompression="Zstd">
         <dataRegionConfigurations>
             <dataRegionConfiguration emptyPagesPoolSize="1" evictionThreshold="2" initialSize="3" metricsEnabled="true"
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 74d2688..429cd48 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -344,6 +344,7 @@
             Assert.AreEqual(17, ds.WalSegmentSize);
             Assert.AreEqual("wal-store", ds.WalPath);
             Assert.AreEqual(TimeSpan.FromSeconds(18), ds.WalAutoArchiveAfterInactivity);
+            Assert.AreEqual(TimeSpan.FromSeconds(19), ds.WalForceArchiveTimeout);
             Assert.IsTrue(ds.WriteThrottlingEnabled);
             Assert.AreEqual(DiskPageCompression.Zstd, ds.WalPageCompression);
 
@@ -1033,6 +1034,7 @@
                     ConcurrencyLevel = 1,
                     PageSize = 5 * 1024,
                     WalAutoArchiveAfterInactivity = TimeSpan.FromSeconds(19),
+                    WalForceArchiveTimeout = TimeSpan.FromSeconds(20),
                     WalPageCompression = DiskPageCompression.Lz4,
                     WalPageCompressionLevel = 10,
                     DefaultDataRegionConfiguration = new DataRegionConfiguration
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 5d57aa8..db287ce 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -630,6 +630,8 @@
             Assert.AreEqual(DataStorageConfiguration.DefaultConcurrencyLevel, cfg.ConcurrencyLevel);
             Assert.AreEqual(DataStorageConfiguration.DefaultWalAutoArchiveAfterInactivity,
                 cfg.WalAutoArchiveAfterInactivity);
+            Assert.AreEqual(DataStorageConfiguration.DefaultWalAutoArchiveAfterInactivity,
+                cfg.WalForceArchiveTimeout);
         }
 
         /// <summary>
@@ -889,6 +891,7 @@
                     ConcurrencyLevel = 1,
                     PageSize = 8 * 1024,
                     WalAutoArchiveAfterInactivity = TimeSpan.FromMinutes(5),
+                    WalForceArchiveTimeout = TimeSpan.FromMinutes(6),
                     CheckpointReadLockTimeout = TimeSpan.FromSeconds(9.5),
                     DefaultDataRegionConfiguration = new DataRegionConfiguration
                     {
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
index 2e2fc72..671fd29 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
@@ -200,6 +200,7 @@
             SystemRegionMaxSize = DefaultSystemRegionMaxSize;
             PageSize = DefaultPageSize;
             WalAutoArchiveAfterInactivity = DefaultWalAutoArchiveAfterInactivity;
+            WalForceArchiveTimeout = DefaultWalAutoArchiveAfterInactivity;
             MaxWalArchiveSize = DefaultMaxWalArchiveSize;
             WalPageCompression = DefaultWalPageCompression;
             ConcurrencyLevel = DefaultConcurrencyLevel;
@@ -241,6 +242,7 @@
             PageSize = reader.ReadInt();
             ConcurrencyLevel = reader.ReadInt();
             WalAutoArchiveAfterInactivity = reader.ReadLongAsTimespan();
+            WalForceArchiveTimeout = reader.ReadLongAsTimespan();
             CheckpointReadLockTimeout = reader.ReadTimeSpanNullable();
             WalPageCompression = (DiskPageCompression)reader.ReadInt();
             WalPageCompressionLevel = reader.ReadIntNullable();
@@ -296,6 +298,7 @@
             writer.WriteInt(PageSize);
             writer.WriteInt(ConcurrencyLevel);
             writer.WriteTimeSpanAsLong(WalAutoArchiveAfterInactivity);
+            writer.WriteTimeSpanAsLong(WalForceArchiveTimeout);
             writer.WriteTimeSpanAsLongNullable(CheckpointReadLockTimeout);
             writer.WriteInt((int)WalPageCompression);
             writer.WriteIntNullable(WalPageCompressionLevel);
@@ -501,6 +504,12 @@
         public TimeSpan WalAutoArchiveAfterInactivity { get; set; }
 
         /// <summary>
+        /// Gets or sets the time for running auto archiving for incompletely WAL segment.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "-00:00:00.001")]
+        public TimeSpan WalForceArchiveTimeout { get; set; }
+
+        /// <summary>
         /// Gets or sets the timeout for checkpoint read lock acquisition.
         /// </summary>
         public TimeSpan? CheckpointReadLockTimeout { get; set; }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 950b738..462949c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -2060,6 +2060,11 @@
                                 <xs:documentation>Inactivity time after which to run WAL segment auto archiving.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
+                        <xs:attribute name="walForceArchiveTimeout" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Time interval (in milliseconds) for force archiving of incompletely WAL segment.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
                         <xs:attribute name="checkpointReadLockTimeout" type="xs:string">
                             <xs:annotation>
                                 <xs:documentation>
diff --git a/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml b/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml
new file mode 100644
index 0000000..e801c46
--- /dev/null
+++ b/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" />
+
+    <bean id="cdc.cfg" class="org.apache.ignite.cdc.CdcConfiguration" />
+</beans>
diff --git a/modules/spring/src/test/config/cdc/correct-cdc-config.xml b/modules/spring/src/test/config/cdc/correct-cdc-config.xml
new file mode 100644
index 0000000..7ee68cb
--- /dev/null
+++ b/modules/spring/src/test/config/cdc/correct-cdc-config.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="dataStorageConfiguration">
+            <bean class="org.apache.ignite.configuration.DataStorageConfiguration">
+                <property name="defaultDataRegionConfiguration">
+                    <bean class="org.apache.ignite.configuration.DataRegionConfiguration">
+                        <property name="persistenceEnabled" value="true" />
+                    </bean>
+                </property>
+                <property name="cdcEnabled" value="true" />
+            </bean>
+        </property>
+    </bean>
+
+    <bean id="cdc.cfg" class="org.apache.ignite.cdc.CdcConfiguration">
+        <property name="consumer">
+            <bean class="org.apache.ignite.cdc.CdcConfigurationTest$TestCdcConsumer" />
+        </property>
+    </bean>
+
+    <bean id="springString" class="java.lang.String">
+        <constructor-arg type="String" value="someString" />
+    </bean>
+
+    <bean id="springString2" class="java.lang.String">
+        <constructor-arg type="String" value="someString2" />
+    </bean>
+</beans>
diff --git a/modules/spring/src/test/config/cdc/double-cdc-config.xml b/modules/spring/src/test/config/cdc/double-cdc-config.xml
new file mode 100644
index 0000000..c99a241
--- /dev/null
+++ b/modules/spring/src/test/config/cdc/double-cdc-config.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <import resource="correct-cdc-config.xml" />
+
+    <bean id="cdc.cfg2" class="org.apache.ignite.cdc.CdcConfiguration" />
+</beans>
diff --git a/modules/spring/src/test/config/cdc/double-ignite-config.xml b/modules/spring/src/test/config/cdc/double-ignite-config.xml
new file mode 100644
index 0000000..26eb8d5
--- /dev/null
+++ b/modules/spring/src/test/config/cdc/double-ignite-config.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <import resource="correct-cdc-config.xml" />
+
+    <bean id="grid.cfg2" class="org.apache.ignite.configuration.IgniteConfiguration" />
+</beans>
diff --git a/modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java b/modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java
new file mode 100644
index 0000000..1ed6244
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.cdc;
+
+import java.util.Iterator;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.cdc.WalRecordsConsumer;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.resources.SpringApplicationContextResource;
+import org.apache.ignite.resources.SpringResource;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.springframework.context.ApplicationContext;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cdc.CdcLoader.loadCdc;
+import static org.apache.ignite.internal.cdc.CdcMain.ERR_MSG;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/** */
+public class CdcConfigurationTest extends GridCommonAbstractTest {
+    /** */
+    @Test
+    public void testLoadConfig() throws Exception {
+        assertThrows(
+            null,
+            () -> loadCdc("modules/spring/src/test/config/cdc/double-ignite-config.xml"),
+            IgniteCheckedException.class,
+            "Exact 1 IgniteConfiguration should be defined. Found 2"
+        );
+
+        assertThrows(
+            null,
+            () -> loadCdc("modules/spring/src/test/config/cdc/double-cdc-config.xml"),
+            IgniteCheckedException.class,
+            "Exact 1 CaptureDataChangeConfiguration configuration should be defined. Found 2"
+        );
+
+        CdcMain cdc =
+            loadCdc("modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml");
+
+        assertNotNull(cdc);
+
+        assertThrows(null, cdc::run, IgniteException.class, ERR_MSG);
+    }
+
+    /** */
+    @Test
+    public void testInjectResources() throws Exception {
+        CdcMain cdc =
+            loadCdc("modules/spring/src/test/config/cdc/correct-cdc-config.xml");
+
+        try (IgniteEx ign = startGrid((IgniteConfiguration)getFieldValue(cdc, "igniteCfg"))) {
+            TestCdcConsumer cnsmr =
+                (TestCdcConsumer)((WalRecordsConsumer<?, ?>)getFieldValue(cdc, "consumer")).consumer();
+
+            assertNotNull(cnsmr);
+
+            CountDownLatch startLatch = cnsmr.startLatch;
+
+            IgniteInternalFuture<?> fut = runAsync(cdc::run);
+
+            startLatch.await(getTestTimeout(), MILLISECONDS);
+
+            assertEquals("someString", cnsmr.springString);
+            assertEquals("someString2", cnsmr.springString2);
+            assertNotNull(cnsmr.log);
+            assertNotNull(cnsmr.ctx);
+
+            fut.cancel();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        cleanPersistenceDir();
+    }
+
+    /** */
+    public static class TestCdcConsumer implements CdcConsumer {
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** */
+        @SpringResource(resourceName = "springString")
+        private String springString;
+
+        /** */
+        private String springString2;
+
+        /** */
+        @SpringApplicationContextResource
+        private ApplicationContext ctx;
+
+        /** */
+        public CountDownLatch startLatch = new CountDownLatch(1);
+
+        /** {@inheritDoc} */
+        @Override public void start() {
+            springString2 = ctx.getBean("springString2", String.class);
+
+            startLatch.countDown();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onEvents(Iterator<CdcEvent> events) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() {
+            // No-Op.
+        }
+    }
+}
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 07222d1..a50d0d7 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -22,6 +22,7 @@
 import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlSelfTest;
 import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlWithSqlEscapeSelfTest;
 import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest;
+import org.apache.ignite.cdc.CdcConfigurationTest;
 import org.apache.ignite.cluster.ClusterStateXmlPropertiesTest;
 import org.apache.ignite.encryption.SpringEncryptedCacheRestartClientTest;
 import org.apache.ignite.encryption.SpringEncryptedCacheRestartTest;
@@ -81,7 +82,10 @@
 
     GridCacheDhtMultiBackupTest.class,
 
-    ClusterStateXmlPropertiesTest.class
+    ClusterStateXmlPropertiesTest.class,
+
+    // CDC tests.
+    CdcConfigurationTest.class
 })
 public class IgniteSpringTestSuite {
 }
diff --git a/parent/pom.xml b/parent/pom.xml
index 93521de..fa58db7 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -496,6 +496,10 @@
                                 <title>Tracing SPI</title>
                                 <packages>org.apache.ignite.spi.tracing*</packages>
                             </group>
+                            <group>
+                                <title>Capture Data Change API</title>
+                                <packages>org.apache.ignite.cdc*</packages>
+                            </group>
                         </groups>
                         <bottom>
                             <![CDATA[