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

[ignite] branch master updated: IGNITE-15246 Fix of an error when starting a node due to exceeding the DataStorageConfiguration#getMaxWalArchiveSize (#9299)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d015707  IGNITE-15246 Fix of an error when starting a node due to exceeding the DataStorageConfiguration#getMaxWalArchiveSize (#9299)
d015707 is described below

commit d015707975f8c2b80448bdbbe7fb1079e4cb98bd
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Thu Aug 5 09:12:25 2021 +0300

    IGNITE-15246 Fix of an error when starting a node due to exceeding the DataStorageConfiguration#getMaxWalArchiveSize (#9299)
---
 .../pagemem/wal/IgniteWriteAheadLogManager.java    |   5 +
 .../GridCacheDatabaseSharedManager.java            |   2 +
 .../persistence/wal/FileWriteAheadLogManager.java  |   7 +-
 .../wal/aware/SegmentArchiveSizeStorage.java       |  90 ++++++++++++-----
 .../cache/persistence/wal/aware/SegmentAware.java  |   7 ++
 .../ignite/cache/AbstractReleaseSegmentTest.java   | 107 +++++++++++++++++++++
 .../ignite/cache/AutoReleaseSegmentSelfTest.java   |  68 +++++++++++++
 .../ReleaseSegmentOnHistoricalRebalanceTest.java   |  79 +--------------
 .../cache/persistence/pagemem/NoOpWALManager.java  |   5 +
 .../persistence/wal/aware/SegmentAwareTest.java    |  48 +++++++++
 .../ignite/testsuites/IgnitePdsTestSuite4.java     |   2 +
 11 files changed, 319 insertions(+), 101 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java
index 044a79a..98d99d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/IgniteWriteAheadLogManager.java
@@ -226,4 +226,9 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni
      * @return Last written pointer.
      */
     WALPointer lastWritePointer();
+
+    /**
+     * Start automatically releasing segments when reaching {@link DataStorageConfiguration#getMaxWalArchiveSize()}.
+     */
+    void startAutoReleaseSegments();
 }
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 22eeef4..d895154 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
@@ -1966,6 +1966,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
             cctx.tm().clearUncommitedStates();
 
+            cctx.wal().startAutoReleaseSegments();
+
             if (recoveryVerboseLogging && log.isInfoEnabled()) {
                 log.info("Partition states information after LOGICAL RECOVERY phase:");
 
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 cca29c7..7e3fde2 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
@@ -3154,7 +3154,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      *
      * @return Size in bytes.
      */
-    public long totalSize(FileDescriptor... fileDescriptors) {
+    public static long totalSize(FileDescriptor... fileDescriptors) {
         long len = 0;
 
         for (FileDescriptor descriptor : fileDescriptors)
@@ -3576,4 +3576,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         return max == UNLIMITED_WAL_ARCHIVE ? max : min != HALF_MAX_WAL_ARCHIVE_SIZE ? min :
             percentage == -1 ? max / 2 : (long)(max * percentage);
     }
+
+    /** {@inheritDoc} */
+    @Override public void startAutoReleaseSegments() {
+        segmentAware.startAutoReleaseSegments();
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java
index 82d1b15..cee0451 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentArchiveSizeStorage.java
@@ -22,6 +22,7 @@ import java.util.TreeMap;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
@@ -50,6 +51,9 @@ class SegmentArchiveSizeStorage {
     /** WAL archive size unlimited. */
     private final boolean walArchiveUnlimited;
 
+    /** Automatically release segments. Guarded by {@code this}. */
+    private boolean autoRelease;
+
     /**
      * Segment sizes. Mapping: segment idx -> size in bytes. Guarded by {@code this}.
      * {@code null} if {@link #walArchiveUnlimited} == {@code true}.
@@ -103,8 +107,7 @@ class SegmentArchiveSizeStorage {
      *                   or negative (e.g. when it is removed from the archive).
      */
     void changeSize(long idx, long sizeChange) {
-        long releaseIdx = -1;
-        int releaseCnt = 0;
+        T2<Long, Integer> forceReleaseSegments = null;
 
         synchronized (this) {
             walArchiveSize += sizeChange;
@@ -118,31 +121,14 @@ class SegmentArchiveSizeStorage {
             }
 
             if (sizeChange > 0) {
-                if (!walArchiveUnlimited && walArchiveSize >= maxWalArchiveSize) {
-                    long size = 0;
-
-                    for (Map.Entry<Long, Long> e : segmentSizes.entrySet()) {
-                        releaseIdx = e.getKey();
-                        releaseCnt++;
-
-                        if (walArchiveSize - (size += e.getValue()) < minWalArchiveSize)
-                            break;
-                    }
-                }
+                forceReleaseSegments = calcForceReleaseSegments();
 
                 notifyAll();
             }
         }
 
-        if (releaseIdx != -1) {
-            if (log.isInfoEnabled()) {
-                log.info("Maximum size of the WAL archive exceeded, the segments will be forcibly released [" +
-                    "maxWalArchiveSize=" + U.humanReadableByteCount(maxWalArchiveSize) + ", releasedSegmentCnt=" +
-                    releaseCnt + ", lastReleasedSegmentIdx=" + releaseIdx + ']');
-            }
-
-            reservationStorage.forceRelease(releaseIdx);
-        }
+        if (forceReleaseSegments != null)
+            forceReleaseSegments(forceReleaseSegments.get1(), forceReleaseSegments.get2());
     }
 
     /**
@@ -214,4 +200,64 @@ class SegmentArchiveSizeStorage {
             }
         }
     }
+
+    /**
+     * Start automatically releasing segments when reaching {@link DataStorageConfiguration#getMaxWalArchiveSize()}.
+     */
+    void startAutoReleaseSegments() {
+        if (!walArchiveUnlimited) {
+            T2<Long, Integer> forceReleaseSegments = null;
+
+            synchronized (this) {
+                autoRelease = true;
+
+                forceReleaseSegments = calcForceReleaseSegments();
+            }
+
+            if (forceReleaseSegments != null)
+                forceReleaseSegments(forceReleaseSegments.get1(), forceReleaseSegments.get2());
+        }
+    }
+
+    /**
+     * Calculation of the segments for which the forced release of the segments will be performed.
+     *
+     * @return Pair: Absolute segment index up (and including) to which the segments will be released, segment count.
+     */
+    @Nullable private synchronized T2<Long, Integer> calcForceReleaseSegments() {
+        if (!walArchiveUnlimited && autoRelease && walArchiveSize >= maxWalArchiveSize) {
+            long releaseIdx = -1;
+            int releaseCnt = 0;
+
+            long size = 0;
+
+            for (Map.Entry<Long, Long> e : segmentSizes.entrySet()) {
+                releaseIdx = e.getKey();
+                releaseCnt++;
+
+                if (walArchiveSize - (size += e.getValue()) < minWalArchiveSize)
+                    break;
+            }
+
+            return releaseIdx == -1 ? null : new T2<>(releaseIdx, releaseCnt);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Forces the release of reserved segments.
+     *
+     * @param absIdx Absolute segment index up (and including) to which the segments will be released.
+     * @param cnt Segment count.
+     */
+    private void forceReleaseSegments(long absIdx, int cnt) {
+        if (log.isInfoEnabled()) {
+            log.info("Maximum size of the WAL archive exceeded, the segments will be forcibly released [" +
+                "maxWalArchiveSize=" + U.humanReadableByteCount(maxWalArchiveSize) + ", releasedSegmentCnt=" +
+                cnt + ", lastReleasedSegmentIdx=" + absIdx + ']');
+        }
+
+        reservationStorage.forceRelease(absIdx);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java
index ee43249..728bd74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAware.java
@@ -384,4 +384,11 @@ public class SegmentAware {
     public long awaitAvailableTruncateArchive() throws IgniteInterruptedCheckedException {
         return truncateStorage.awaitAvailableTruncate();
     }
+
+    /**
+     * Start automatically releasing segments when reaching {@link DataStorageConfiguration#getMaxWalArchiveSize()}.
+     */
+    public void startAutoReleaseSegments() {
+        archiveSizeStorage.startAutoReleaseSegments();
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/AbstractReleaseSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/cache/AbstractReleaseSegmentTest.java
new file mode 100644
index 0000000..5009c49
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/AbstractReleaseSegmentTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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 org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+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.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.aware.SegmentAware;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
+
+/**
+ * Base class for testing the release of segments when performing an operation.
+ */
+public abstract class AbstractReleaseSegmentTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setFailureHandler(new StopNodeFailureHandler())
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setWalSegmentSize((int)(2 * U.MB))
+                    .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))
+            ).setCacheConfiguration(
+                new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+                    .setAffinity(new RendezvousAffinityFunction(false, 2))
+                    .setBackups(1)
+            );
+    }
+
+    /**
+     * Populates the given cache and forces a new checkpoint every 100 updates.
+     *
+     * @param cache Cache.
+     * @param cnt Entry count.
+     * @param o Key offset.
+     * @throws Exception If failed.
+     */
+    protected void populate(IgniteCache<Integer, ? super Object> cache, int cnt, int o) throws Exception {
+        for (int i = 0; i < cnt; i++) {
+            if (i % 100 == 0)
+                forceCheckpoint();
+
+            cache.put(i + o, new byte[64 * 1024]);
+        }
+    }
+
+    /**
+     * Releases WAL segment.
+     *
+     * @param n Node.
+     * @param reserved Reserved segment.
+     */
+    protected void release(IgniteEx n, @Nullable WALPointer reserved) {
+        while (reserved != null && walMgr(n).reserved(reserved))
+            walMgr(n).release(reserved);
+    }
+
+    /**
+     * Returns an instance of {@link SegmentAware} for the given ignite node.
+     *
+     * @return Segment aware.
+     */
+    protected SegmentAware segmentAware(IgniteEx n) {
+        return getFieldValue(walMgr(n), "segmentAware");
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/AutoReleaseSegmentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/AutoReleaseSegmentSelfTest.java
new file mode 100644
index 0000000..f7f04aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/AutoReleaseSegmentSelfTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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 org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.totalSize;
+
+/**
+ * Class for testing automatic release of segments.
+ */
+public class AutoReleaseSegmentSelfTest extends AbstractReleaseSegmentTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.getDataStorageConfiguration()
+            .setWalSegmentSize((int)(2 * U.MB))
+            .setMaxWalArchiveSize(10 * U.MB);
+
+        return cfg;
+    }
+
+    /**
+     * Checking that if at the time of start the node, the {@link DataStorageConfiguration#getMaxWalArchiveSize()}
+     * is exceeded, then there will be no automatic release of segments due to which there will be an error in
+     * {@code GridCacheDatabaseSharedManager#applyLogicalUpdates}.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testStartNodeWithExceedMaxWalArchiveSize() throws Exception {
+        IgniteEx n = startGrid(0);
+
+        n.cluster().state(ACTIVE);
+
+        forceCheckpoint();
+        enableCheckpoints(n, false);
+
+        int i = 0;
+
+        while (totalSize(walMgr(n).walArchiveFiles()) < 20 * U.MB)
+            n.cache(DEFAULT_CACHE_NAME).put(i++, new byte[(int)(64 * U.KB)]);
+
+        stopGrid(0);
+        startGrid(0);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ReleaseSegmentOnHistoricalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ReleaseSegmentOnHistoricalRebalanceTest.java
index e82579a..9d1cc64 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/ReleaseSegmentOnHistoricalRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/ReleaseSegmentOnHistoricalRebalanceTest.java
@@ -18,13 +18,6 @@
 package org.apache.ignite.cache;
 
 import java.lang.reflect.Method;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-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.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -40,8 +33,6 @@ import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.Nullable;
 import org.junit.Test;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
@@ -58,38 +49,7 @@ import static org.mockito.Mockito.when;
  * Testing the release of WAL segments during historical rebalance.
  */
 @WithSystemProperty(key = IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value = "0")
-public class ReleaseSegmentOnHistoricalRebalanceTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        stopAllGrids();
-        cleanPersistenceDir();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        stopAllGrids();
-        cleanPersistenceDir();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        return super.getConfiguration(igniteInstanceName)
-            .setFailureHandler(new StopNodeFailureHandler())
-            .setDataStorageConfiguration(
-                new DataStorageConfiguration()
-                    .setWalSegmentSize((int)(2 * U.MB))
-                    .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))
-            ).setCacheConfiguration(
-                new CacheConfiguration<>(DEFAULT_CACHE_NAME)
-                    .setAffinity(new RendezvousAffinityFunction(false, 2))
-                    .setBackups(1)
-            );
-    }
-
+public class ReleaseSegmentOnHistoricalRebalanceTest extends AbstractReleaseSegmentTest {
     /**
      * Checks that if release the segment after {@link CheckpointHistory#searchAndReserveCheckpoints},
      * there will be no errors and the rebalance will be completed.
@@ -273,23 +233,6 @@ public class ReleaseSegmentOnHistoricalRebalanceTest extends GridCommonAbstractT
     }
 
     /**
-     * Populates the given cache and forces a new checkpoint every 100 updates.
-     *
-     * @param cache Cache.
-     * @param cnt Entry count.
-     * @param o Key offset.
-     * @throws Exception If failed.
-     */
-    private void populate(IgniteCache<Integer, ? super Object> cache, int cnt, int o) throws Exception {
-        for (int i = 0; i < cnt; i++) {
-            if (i % 100 == 0)
-                forceCheckpoint();
-
-            cache.put(i + o, new byte[64 * 1024]);
-        }
-    }
-
-    /**
      * Sets the spy to {@code CheckpointMarkersStorage#cpHistory}.
      *
      * @param n Node.
@@ -332,26 +275,6 @@ public class ReleaseSegmentOnHistoricalRebalanceTest extends GridCommonAbstractT
     }
 
     /**
-     * Releases WAL segment.
-     *
-     * @param n Node.
-     * @param reserved Reserved segment.
-     */
-    private void release(IgniteEx n, @Nullable WALPointer reserved) {
-        while (reserved != null && walMgr(n).reserved(reserved))
-            walMgr(n).release(reserved);
-    }
-
-    /**
-     * Returns an instance of {@link SegmentAware} for the given ignite node.
-     *
-     * @return Segment aware.
-     */
-    private SegmentAware segmentAware(IgniteEx n) {
-        return getFieldValue(walMgr(n), "segmentAware");
-    }
-
-    /**
      * Returns an instance of {@link GridCacheOffheapManager} for the given ignite node.
      *
      * @param cache Cache.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
index 29a9fc7..eccef97 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
@@ -196,4 +196,9 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager {
     @Override public WALPointer lastWritePointer() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public void startAutoReleaseSegments() {
+        // No-op.
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java
index 1652d35..d95fd88 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentAwareTest.java
@@ -878,6 +878,8 @@ public class SegmentAwareTest {
         SegmentAware aware = segmentAware(1, false, 50, 100);
         SegmentReservationStorage reservationStorage = reservationStorage(aware);
 
+        aware.startAutoReleaseSegments();
+
         for (int i = 0; i < 9; i++)
             aware.addSize(i, 10);
 
@@ -930,6 +932,52 @@ public class SegmentAwareTest {
     }
 
     /**
+     * Checking that when the {@code SegmentArchiveSizeStorage#maxWalArchiveSize} is reached
+     * and after calling the {@link SegmentAware#startAutoReleaseSegments()}
+     * the segments will be released to the {@code SegmentArchiveSizeStorage#minWalArchiveSize},
+     * and it will also not be possible to reserve them.
+     */
+    @Test
+    public void testReleaseSegmentsOnExceedMaxWalArchiveSizeAfterStartAutoReleaseSegments() {
+        SegmentAware aware = segmentAware(1, false, 50, 100);
+        SegmentReservationStorage reservationStorage = reservationStorage(aware);
+
+        for (int i = 0; i < 9; i++)
+            aware.addSize(i, 10);
+
+        assertTrue(aware.reserve(0));
+        assertTrue(aware.reserve(1));
+        assertTrue(aware.reserve(8));
+
+        aware.addSize(9, 10);
+
+        assertTrue(aware.reserved(0));
+        assertTrue(aware.reserved(1));
+        assertTrue(aware.reserved(8));
+        assertEquals(-1, reservationStorage.minReserveIdx());
+
+        aware.startAutoReleaseSegments();
+
+        assertFalse(aware.reserved(0));
+        assertFalse(aware.reserved(1));
+        assertTrue(aware.reserved(8));
+        assertEquals(5, reservationStorage.minReserveIdx());
+
+        for (int i = 0; i <= 5; i++) {
+            assertFalse(aware.reserve(i));
+            assertFalse(aware.reserved(i));
+
+            assertTrue(aware.minReserveIndex(i));
+        }
+
+        for (int i = 6; i < 10; i++) {
+            assertTrue(aware.reserve(i));
+
+            assertFalse(aware.minReserveIndex(i));
+        }
+    }
+
+    /**
      * Assert that future is still not finished.
      *
      * @param future Future to check.
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
index f288e38..c0972b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import org.apache.ignite.cache.AutoReleaseSegmentSelfTest;
 import org.apache.ignite.cache.BreakRebalanceChainTest;
 import org.apache.ignite.cache.CircledRebalanceTest;
 import org.apache.ignite.cache.NoUnnecessaryRebalancesTest;
@@ -111,6 +112,7 @@ public class IgnitePdsTestSuite4 {
         GridTestUtils.addTestIfNeeded(suite, IgnitePdsSpuriousRebalancingOnNodeJoinTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, RebalanceCompleteDuringExchangeTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, ReleaseSegmentOnHistoricalRebalanceTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, AutoReleaseSegmentSelfTest.class, ignoredTests);
 
         // Page lock tracker tests.
         GridTestUtils.addTestIfNeeded(suite, PageLockTrackerManagerTest.class, ignoredTests);