You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2020/12/17 14:56:14 UTC

[ignite] branch master updated: IGNITE-13848 Fixed incorrect updating of SegmentReservationStorage#minReserveIdx when truncating WAL segments. Fixes #8573

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

sk0x50 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 54ef8b8  IGNITE-13848 Fixed incorrect updating of SegmentReservationStorage#minReserveIdx when truncating WAL segments. Fixes #8573
54ef8b8 is described below

commit 54ef8b804a6e9ec4edd95a367e19ebb768c36b07
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Thu Dec 17 17:55:37 2020 +0300

    IGNITE-13848 Fixed incorrect updating of SegmentReservationStorage#minReserveIdx when truncating WAL segments. Fixes #8573
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 .../persistence/wal/FileWriteAheadLogManager.java  | 66 ++++++++--------------
 .../cache/persistence/wal/aware/SegmentAware.java  |  6 +-
 .../wal/aware/SegmentCompressStorage.java          | 17 +++++-
 .../db/IgnitePdsReserveWalSegmentsTest.java        | 28 ++++++++-
 .../persistence/wal/aware/SegmentAwareTest.java    | 61 ++++++++++----------
 5 files changed, 101 insertions(+), 77 deletions(-)

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 de27753..f22aed9 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
@@ -169,46 +169,31 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     public static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp");
 
     /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */
-    public static final FileFilter WAL_SEGMENT_FILE_FILTER = new FileFilter() {
-        @Override public boolean accept(File file) {
-            return !file.isDirectory() && WAL_NAME_PATTERN.matcher(file.getName()).matches();
-        }
-    };
+    public static final FileFilter WAL_SEGMENT_FILE_FILTER = file -> !file.isDirectory() &&
+        WAL_NAME_PATTERN.matcher(file.getName()).matches();
 
     /** WAL segment temporary file filter, see {@link #WAL_TEMP_NAME_PATTERN} */
-    private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = new FileFilter() {
-        @Override public boolean accept(File file) {
-            return !file.isDirectory() && WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches();
-        }
-    };
+    private static final FileFilter WAL_SEGMENT_TEMP_FILE_FILTER = file -> !file.isDirectory() &&
+        WAL_TEMP_NAME_PATTERN.matcher(file.getName()).matches();
 
     /** */
     public static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip");
 
     /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */
-    public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = new FileFilter() {
-        @Override public boolean accept(File file) {
-            return !file.isDirectory() && (WAL_NAME_PATTERN.matcher(file.getName()).matches() ||
-                WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches());
-        }
-    };
+    public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = file -> !file.isDirectory() &&
+        (WAL_NAME_PATTERN.matcher(file.getName()).matches() ||
+            WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches());
 
     /** */
     private static final Pattern WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip\\.tmp");
 
     /** */
-    private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = new FileFilter() {
-        @Override public boolean accept(File file) {
-            return !file.isDirectory() && WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches();
-        }
-    };
+    private static final FileFilter WAL_SEGMENT_FILE_COMPACTED_FILTER = file -> !file.isDirectory() &&
+        WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches();
 
     /** */
-    private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = new FileFilter() {
-        @Override public boolean accept(File file) {
-            return !file.isDirectory() && WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches();
-        }
-    };
+    private static final FileFilter WAL_SEGMENT_TEMP_FILE_COMPACTED_FILTER = file -> !file.isDirectory() &&
+        WAL_SEGMENT_TEMP_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches();
 
     /** Buffer size. */
     private static final int BUF_SIZE = 1024 * 1024;
@@ -497,7 +482,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 });
             }
 
-            segmentAware = new SegmentAware(dsCfg.getWalSegments(), dsCfg.isWalCompactionEnabled());
+            segmentAware = new SegmentAware(dsCfg.getWalSegments(), dsCfg.isWalCompactionEnabled(), log);
 
             // We have to initialize compressor before archiver in order to setup already compressed segments.
             // Otherwise, FileArchiver initialization will trigger redundant work for FileCompressor.
@@ -1062,26 +1047,25 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             if (desc.idx >= lastCheckpointPtr.index() // We cannot delete segments needed for binary recovery.
                 || desc.idx >= lastArchived // We cannot delete last segment, it is needed at start of node and avoid gaps.
+                || desc.idx >= high.index() // We cannot delete segments larger than the border.
                 || !segmentAware.minReserveIndex(desc.idx)) // We cannot delete reserved segment.
                 return deleted;
 
-            if (desc.idx < high.index()) {
-                if (!desc.file.delete()) {
-                    U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " +
-                        desc.file.getAbsolutePath());
-                }
-                else {
-                    deleted++;
+            if (!desc.file.delete()) {
+                U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " +
+                    desc.file.getAbsolutePath());
+            }
+            else {
+                deleted++;
 
-                    segmentSize.remove(desc.idx());
-                }
+                segmentSize.remove(desc.idx());
+            }
 
-                // Bump up the oldest archive segment index.
-                if (segmentAware.lastTruncatedArchiveIdx() < desc.idx)
-                    segmentAware.lastTruncatedArchiveIdx(desc.idx);
+            // Bump up the oldest archive segment index.
+            if (segmentAware.lastTruncatedArchiveIdx() < desc.idx)
+                segmentAware.lastTruncatedArchiveIdx(desc.idx);
 
-                cctx.kernalContext().encryption().onWalSegmentRemoved(desc.idx);
-            }
+            cctx.kernalContext().encryption().onWalSegmentRemoved(desc.idx);
         }
 
         return deleted;
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 89523db..8d676c9 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.wal.aware;
 
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 
 /**
@@ -43,12 +44,13 @@ public class SegmentAware {
      *
      * @param walSegmentsCnt Total WAL segments count.
      * @param compactionEnabled Is wal compaction enabled.
+     * @param log Logger.
      */
-    public SegmentAware(int walSegmentsCnt, boolean compactionEnabled) {
+    public SegmentAware(int walSegmentsCnt, boolean compactionEnabled, IgniteLogger log) {
         segmentArchivedStorage = new SegmentArchivedStorage(segmentLockStorage);
 
         segmentCurrStateStorage = new SegmentCurrentStateStorage(walSegmentsCnt);
-        segmentCompressStorage = new SegmentCompressStorage(compactionEnabled);
+        segmentCompressStorage = new SegmentCompressStorage(log, compactionEnabled);
 
         segmentArchivedStorage.addObserver(segmentCurrStateStorage::onSegmentArchived);
         segmentArchivedStorage.addObserver(segmentCompressStorage::onSegmentArchived);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java
index 62fe69d..5fca1ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/aware/SegmentCompressStorage.java
@@ -21,12 +21,16 @@ import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Queue;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 
 /**
  * Storage of actual information about current index of compressed segments.
  */
 public class SegmentCompressStorage {
+    /** Logger. */
+    private final IgniteLogger log;
+
     /** Flag of interrupt waiting on this object. */
     private volatile boolean interrupted;
 
@@ -51,9 +55,11 @@ public class SegmentCompressStorage {
     /**
      * Constructor.
      *
+     * @param log Logger.
      * @param compactionEnabled If WAL compaction enabled.
      */
-    SegmentCompressStorage(boolean compactionEnabled) {
+    SegmentCompressStorage(IgniteLogger log, boolean compactionEnabled) {
+        this.log = log;
         this.compactionEnabled = compactionEnabled;
     }
 
@@ -63,6 +69,9 @@ public class SegmentCompressStorage {
      * @param compressedIdx Index of compressed segment.
      */
     synchronized void onSegmentCompressed(long compressedIdx) {
+        if (log.isInfoEnabled())
+            log.info("Segment compressed notification [idx=" + compressedIdx + ']');
+
         if (compressedIdx > lastMaxCompressedIdx)
             lastMaxCompressedIdx = compressedIdx;
 
@@ -129,8 +138,12 @@ public class SegmentCompressStorage {
      * Callback for waking up compressor when new segment is archived.
      */
     synchronized void onSegmentArchived(long lastAbsArchivedIdx) {
-        while (lastEnqueuedToCompressIdx < lastAbsArchivedIdx && compactionEnabled)
+        while (lastEnqueuedToCompressIdx < lastAbsArchivedIdx && compactionEnabled) {
+            if (log.isInfoEnabled())
+                log.info("Enqueuing segment for compression [idx=" + (lastEnqueuedToCompressIdx + 1) + ']');
+
             segmentsToCompress.add(++lastEnqueuedToCompressIdx);
+        }
 
         notifyAll();
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
index c18f062..48ae33e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
@@ -28,13 +28,14 @@ import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
-import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValueHierarchy;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 
 /**
@@ -181,6 +182,29 @@ public class IgnitePdsReserveWalSegmentsTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Check that the minimum reserved index will not be greater than the actual deleted segment.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMinReserveIdx() throws Exception {
+        IgniteEx n = prepareGrid(1);
+
+        forceCheckpoint();
+
+        FileWriteAheadLogManager wal = (FileWriteAheadLogManager)n.context().cache().context().wal();
+        assertNotNull(wal);
+
+        if (compactionEnabled(n))
+            assertTrue(waitForCondition(() -> wal.lastCompactedSegment() >= 1, 10_000));
+
+        assertEquals(1, wal.truncate(new WALPointer(1, 0, 0)));
+
+        Long minReserveIdx = getFieldValueHierarchy(wal, "segmentAware", "reservationStorage", "minReserveIdx");
+        assertEquals(0L, minReserveIdx.longValue());
+    }
+
+    /**
      * Starts grid and populates test data.
      *
      * @param cnt Grid count.
@@ -211,7 +235,7 @@ public class IgnitePdsReserveWalSegmentsTest extends GridCommonAbstractTest {
      * @param dbMgr Database shared manager.
      */
     private long getReservedWalSegmentIndex(IgniteWriteAheadLogManager dbMgr) {
-        return ((WALPointer)GridTestUtils.getFieldValueHierarchy(dbMgr, "lastCheckpointPtr")).index();
+        return ((WALPointer)getFieldValueHierarchy(dbMgr, "lastCheckpointPtr")).index();
     }
 
     /**
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 60663ef..5037ca3 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
@@ -21,6 +21,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.junit.Test;
 
@@ -43,7 +44,7 @@ public class SegmentAwareTest {
      */
     @Test
     public void testAvoidDeadlockArchiverAndLockStorage() throws IgniteCheckedException {
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         int iterationCnt = 100_000;
         int segmentToHandle = 1;
@@ -80,7 +81,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishAwaitSegment_WhenExactWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5));
 
@@ -97,7 +98,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishAwaitSegment_WhenGreaterThanWaitingSegmentWasSet() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5));
 
@@ -114,7 +115,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishAwaitSegment_WhenNextSegmentEqualToWaitingOne() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5));
 
@@ -137,7 +138,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishAwaitSegment_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegment(5));
 
@@ -154,7 +155,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentForArchive_WhenWorkSegmentIncremented() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.curAbsWalIdx(5);
         aware.setLastArchivedAbsoluteIndex(4);
@@ -174,7 +175,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentForArchive_WhenWorkSegmentGreaterValue() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.curAbsWalIdx(5);
         aware.setLastArchivedAbsoluteIndex(4);
@@ -194,7 +195,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentForArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.curAbsWalIdx(5);
         aware.setLastArchivedAbsoluteIndex(4);
@@ -214,7 +215,7 @@ public class SegmentAwareTest {
     @Test
     public void testCorrectCalculateNextSegmentIndex() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.curAbsWalIdx(5);
 
@@ -231,7 +232,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitNextAbsoluteIndex_WhenMarkAsArchivedFirstSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(2, false);
+        SegmentAware aware = new SegmentAware(2, false, new NullLogger());
 
         aware.curAbsWalIdx(1);
         aware.setLastArchivedAbsoluteIndex(-1);
@@ -251,7 +252,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitNextAbsoluteIndex_WhenSetToArchivedFirst() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(2, false);
+        SegmentAware aware = new SegmentAware(2, false, new NullLogger());
 
         aware.curAbsWalIdx(1);
         aware.setLastArchivedAbsoluteIndex(-1);
@@ -271,7 +272,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitNextAbsoluteIndex_WhenOnlyForceInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(2, false);
+        SegmentAware aware = new SegmentAware(2, false, new NullLogger());
 
         aware.curAbsWalIdx(2);
         aware.setLastArchivedAbsoluteIndex(-1);
@@ -297,7 +298,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishSegmentArchived_WhenSetExactWaitingSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5));
 
@@ -314,7 +315,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishSegmentArchived_WhenMarkExactWaitingSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5));
 
@@ -331,7 +332,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishSegmentArchived_WhenSetGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5));
 
@@ -348,7 +349,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishSegmentArchived_WhenMarkGreaterThanWaitingSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         IgniteInternalFuture future = awaitThread(() -> aware.awaitSegmentArchived(5));
 
@@ -365,7 +366,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishSegmentArchived_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.curAbsWalIdx(5);
         aware.setLastArchivedAbsoluteIndex(4);
@@ -385,7 +386,7 @@ public class SegmentAwareTest {
     @Test
     public void testMarkAsMovedToArchive_WhenReleaseLockedSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         assertTrue(aware.lock(5));
 
@@ -404,7 +405,7 @@ public class SegmentAwareTest {
     @Test
     public void testMarkAsMovedToArchive_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         assertTrue(aware.lock(5));
 
@@ -426,7 +427,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentToCompress_WhenSetLastArchivedSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, true);
+        SegmentAware aware = new SegmentAware(10, true, new NullLogger());
 
         aware.onSegmentCompressed(5);
 
@@ -445,7 +446,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentToCompress_WhenMarkLastArchivedSegment() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, true);
+        SegmentAware aware = new SegmentAware(10, true, new NullLogger());
 
         aware.onSegmentCompressed(5);
 
@@ -463,7 +464,7 @@ public class SegmentAwareTest {
      */
     @Test
     public void testCorrectCalculateNextCompressSegment() throws IgniteCheckedException, InterruptedException {
-        SegmentAware aware = new SegmentAware(10, true);
+        SegmentAware aware = new SegmentAware(10, true, new NullLogger());
 
         aware.setLastArchivedAbsoluteIndex(6);
 
@@ -477,7 +478,7 @@ public class SegmentAwareTest {
     @Test
     public void testFinishWaitSegmentToCompress_WhenInterruptWasCall() throws IgniteCheckedException, InterruptedException {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, true);
+        SegmentAware aware = new SegmentAware(10, true, new NullLogger());
         aware.onSegmentCompressed(5);
 
         IgniteInternalFuture future = awaitThread(aware::waitNextSegmentToCompress);
@@ -494,7 +495,7 @@ public class SegmentAwareTest {
      */
     @Test
     public void testLastCompressedIdxProperOrdering() throws IgniteInterruptedCheckedException {
-        SegmentAware aware = new SegmentAware(10, true);
+        SegmentAware aware = new SegmentAware(10, true, new NullLogger());
 
         for (int i = 0; i < 5; i++) {
             aware.setLastArchivedAbsoluteIndex(i);
@@ -519,7 +520,7 @@ public class SegmentAwareTest {
     @Test
     public void testReserveCorrectly() {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         // Set limits.
         aware.curAbsWalIdx(10);
@@ -567,7 +568,7 @@ public class SegmentAwareTest {
     @Test
     public void testAssertFail_WhenReleaseUnreservedSegment() {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         aware.reserve(5);
         try {
@@ -587,7 +588,7 @@ public class SegmentAwareTest {
     @Test
     public void testReserveWorkSegmentCorrectly() {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         //when: lock one segment twice.
         assertTrue(aware.lock(5));
@@ -621,7 +622,7 @@ public class SegmentAwareTest {
     @Test
     public void testAssertFail_WhenReleaseUnreservedWorkSegment() {
         //given: thread which awaited segment.
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         assertTrue(aware.lock(5));
         try {
@@ -639,7 +640,7 @@ public class SegmentAwareTest {
      */
     @Test
     public void testReservationBorder() {
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         assertTrue(aware.reserve(0));
         assertTrue(aware.reserve(1));
@@ -661,7 +662,7 @@ public class SegmentAwareTest {
      */
     @Test
     public void testLockBorder() {
-        SegmentAware aware = new SegmentAware(10, false);
+        SegmentAware aware = new SegmentAware(10, false, new NullLogger());
 
         assertTrue(aware.lock(0));
         assertTrue(aware.lock(1));