This is an automated email from the ASF dual-hosted git repository.
apolovtsev pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new 5bc6acef7ca IGNITE-26290 Add max log entry size limit to configuration
(#7255)
5bc6acef7ca is described below
commit 5bc6acef7ca2df9aa606086583ea7ade97b6820f
Author: Alexander Polovtcev <[email protected]>
AuthorDate: Wed Dec 17 09:38:34 2025 +0200
IGNITE-26290 Add max log entry size limit to configuration (#7255)
---
modules/raft-api/build.gradle | 1 +
.../LogStorageConfigurationSchema.java | 32 +++++-
.../LogStorageConfigurationValidator.java | 62 +++++++++++
.../configuration/RaftConfigurationModule.java | 7 ++
.../configuration/RaftConfigurationSchema.java | 1 +
...hema.java => ValidLogStorageConfiguration.java} | 31 ++----
.../LogStorageConfigurationValidatorTest.java | 123 +++++++++++++++++++++
.../raft/storage/segstore/SegmentFileManager.java | 44 +++++---
.../storage/segstore/SegmentFileManagerTest.java | 3 +-
9 files changed, 265 insertions(+), 39 deletions(-)
diff --git a/modules/raft-api/build.gradle b/modules/raft-api/build.gradle
index 1a52af9e58f..124d7eb1e1e 100644
--- a/modules/raft-api/build.gradle
+++ b/modules/raft-api/build.gradle
@@ -31,6 +31,7 @@ dependencies {
implementation libs.auto.service.annotations
testImplementation testFixtures(project(':ignite-core'))
+ testImplementation testFixtures(project(':ignite-configuration'))
annotationProcessor project(":ignite-configuration-annotation-processor")
annotationProcessor libs.auto.service
diff --git
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
index 46173b44353..184bdeafec0 100644
---
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
+++
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
@@ -26,18 +26,42 @@ import org.apache.ignite.configuration.validation.Range;
/** Configuration of the Raft log storage. */
@Config
public class LogStorageConfigurationSchema {
+ public static final int DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE = 10;
+
+ public static final int DEFAULT_SEGMENT_FILE_SIZE_BYTES =
Integer.MAX_VALUE;
+
+ public static final int UNSPECIFIED_MAX_LOG_ENTRY_SIZE = -1;
+
/**
* Maximum size of the log storage checkpoint queue.
*/
@Value(hasDefault = true)
@Range(min = 1)
- public int maxCheckpointQueueSize = 10;
+ public int maxCheckpointQueueSize = DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE;
/**
* Size of a segment file in bytes.
*/
- @SuppressWarnings("PointlessArithmeticExpression") // Suppressed for
better readability.
@Value(hasDefault = true)
- @Range(min = 1 * KiB, max = Integer.MAX_VALUE)
- public long segmentFileSizeBytes = Integer.MAX_VALUE;
+ @Range(min = 4 * KiB, max = Integer.MAX_VALUE)
+ public long segmentFileSizeBytes = DEFAULT_SEGMENT_FILE_SIZE_BYTES;
+
+ /**
+ * Maximum allowed size of a log entry in bytes.
+ */
+ @Value(hasDefault = true)
+ public int maxLogEntrySizeBytes = UNSPECIFIED_MAX_LOG_ENTRY_SIZE;
+
+ /**
+ * Computes the default maximum log entry size based on the segment file
size.
+ *
+ * <p>Should be used to calculate the maximum log entry size when it is
equal to {@link #UNSPECIFIED_MAX_LOG_ENTRY_SIZE}.
+ */
+ @SuppressWarnings("NumericCastThatLosesPrecision")
+ public static int computeDefaultMaxLogEntrySizeBytes(long
segmentFileSizeBytes) {
+ // We set the max entry size to 90% of the segment file to leave some
space for metadata overhead.
+ long maxAllowedEntrySize = (long) (segmentFileSizeBytes * 0.9);
+
+ return (int) Math.min(Integer.MAX_VALUE, maxAllowedEntrySize);
+ }
}
diff --git
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidator.java
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidator.java
new file mode 100644
index 00000000000..4e388adf5d1
--- /dev/null
+++
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.raft.configuration;
+
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.computeDefaultMaxLogEntrySizeBytes;
+
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+
+class LogStorageConfigurationValidator implements
Validator<ValidLogStorageConfiguration, LogStorageView> {
+ static final LogStorageConfigurationValidator INSTANCE = new
LogStorageConfigurationValidator();
+
+ @Override
+ public void validate(ValidLogStorageConfiguration annotation,
ValidationContext<LogStorageView> ctx) {
+ LogStorageView newValue = ctx.getNewValue();
+
+ validateLogEntrySize(newValue, ctx);
+ }
+
+ private static void validateLogEntrySize(LogStorageView config,
ValidationContext<LogStorageView> ctx) {
+ int maxEntrySize = config.maxLogEntrySizeBytes();
+
+ if (maxEntrySize ==
LogStorageConfigurationSchema.UNSPECIFIED_MAX_LOG_ENTRY_SIZE) {
+ return;
+ }
+
+ if (maxEntrySize <= 0) {
+ String errorMsg = String.format("Maximum log entry size must be
positive, got %d.", maxEntrySize);
+
+ ctx.addIssue(new ValidationIssue(ctx.currentKey(), errorMsg));
+
+ return;
+ }
+
+ int maxAllowedEntrySize =
computeDefaultMaxLogEntrySizeBytes(config.segmentFileSizeBytes());
+
+ if (maxEntrySize > maxAllowedEntrySize) {
+ String errorMsg = String.format(
+ "Maximum log entry size is too big (%d bytes), maximum
allowed log entry size is %d bytes.",
+ maxEntrySize, maxAllowedEntrySize
+ );
+
+ ctx.addIssue(new ValidationIssue(ctx.currentKey(), errorMsg));
+ }
+ }
+}
diff --git
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationModule.java
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationModule.java
index 34ebe50aee1..97598edd435 100644
---
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationModule.java
+++
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationModule.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.raft.configuration;
import com.google.auto.service.AutoService;
import java.util.Collection;
import java.util.List;
+import java.util.Set;
import org.apache.ignite.configuration.ConfigurationModule;
import org.apache.ignite.configuration.SuperRootChange;
import org.apache.ignite.configuration.annotation.ConfigurationType;
+import org.apache.ignite.configuration.validation.Validator;
/**
* {@link ConfigurationModule} for node-local configuration provided by
ignite-raft.
@@ -47,6 +49,11 @@ public class RaftConfigurationModule implements
ConfigurationModule {
);
}
+ @Override
+ public Set<Validator<?, ?>> validators() {
+ return Set.of(LogStorageConfigurationValidator.INSTANCE);
+ }
+
@Override
public void migrateDeprecatedConfigurations(SuperRootChange
superRootChange) {
RaftExtensionChange raftExtensionChange =
superRootChange.changeRoot(RaftExtensionConfiguration.KEY);
diff --git
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationSchema.java
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationSchema.java
index 5e7400eb129..0c26ccb3732 100644
---
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationSchema.java
+++
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/RaftConfigurationSchema.java
@@ -108,5 +108,6 @@ public class RaftConfigurationSchema {
// TODO: Uncomment when the new log storage is released, see
https://issues.apache.org/jira/browse/IGNITE-26300.
// @ConfigValue
+ // @ValidLogStorageConfiguration
// public LogStorageConfigurationSchema logStorage;
}
diff --git
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/ValidLogStorageConfiguration.java
similarity index 52%
copy from
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
copy to
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/ValidLogStorageConfiguration.java
index 46173b44353..483832688ed 100644
---
a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationSchema.java
+++
b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/configuration/ValidLogStorageConfiguration.java
@@ -17,27 +17,16 @@
package org.apache.ignite.internal.raft.configuration;
-import static org.apache.ignite.internal.util.Constants.KiB;
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
-import org.apache.ignite.configuration.annotation.Config;
-import org.apache.ignite.configuration.annotation.Value;
-import org.apache.ignite.configuration.validation.Range;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
-/** Configuration of the Raft log storage. */
-@Config
-public class LogStorageConfigurationSchema {
- /**
- * Maximum size of the log storage checkpoint queue.
- */
- @Value(hasDefault = true)
- @Range(min = 1)
- public int maxCheckpointQueueSize = 10;
-
- /**
- * Size of a segment file in bytes.
- */
- @SuppressWarnings("PointlessArithmeticExpression") // Suppressed for
better readability.
- @Value(hasDefault = true)
- @Range(min = 1 * KiB, max = Integer.MAX_VALUE)
- public long segmentFileSizeBytes = Integer.MAX_VALUE;
+/**
+ * Annotations used to mark {@link LogStorageConfiguration} fields to enable
their validation.
+ */
+@Target(FIELD)
+@Retention(RUNTIME)
+public @interface ValidLogStorageConfiguration {
}
diff --git
a/modules/raft-api/src/test/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidatorTest.java
b/modules/raft-api/src/test/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidatorTest.java
new file mode 100644
index 00000000000..9465d77d7ed
--- /dev/null
+++
b/modules/raft-api/src/test/java/org/apache/ignite/internal/raft/configuration/LogStorageConfigurationValidatorTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.raft.configuration;
+
+import static
org.apache.ignite.internal.configuration.validation.TestValidationUtil.mockValidationContext;
+import static
org.apache.ignite.internal.configuration.validation.TestValidationUtil.validate;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.DEFAULT_SEGMENT_FILE_SIZE_BYTES;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.UNSPECIFIED_MAX_LOG_ENTRY_SIZE;
+import static org.mockito.Mockito.mock;
+
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.junit.jupiter.api.Test;
+
+class LogStorageConfigurationValidatorTest extends BaseIgniteAbstractTest {
+ private final LogStorageConfigurationValidator validator = new
LogStorageConfigurationValidator();
+
+ @Test
+ void unspecifiedLogEntrySizeIsValid() {
+ var config = new MockLogStorageView(
+ DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE,
+ DEFAULT_SEGMENT_FILE_SIZE_BYTES,
+ UNSPECIFIED_MAX_LOG_ENTRY_SIZE
+ );
+
+ validate(
+ validator,
+ mock(ValidLogStorageConfiguration.class),
+ mockValidationContext(null, config)
+ );
+ }
+
+ @Test
+ void correctLogEntrySizeIsValid() {
+ var config = new MockLogStorageView(
+ DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE,
+ DEFAULT_SEGMENT_FILE_SIZE_BYTES,
+ (int) (DEFAULT_SEGMENT_FILE_SIZE_BYTES * 0.9)
+ );
+
+ validate(
+ validator,
+ mock(ValidLogStorageConfiguration.class),
+ mockValidationContext(null, config)
+ );
+ }
+
+ @Test
+ void zeroLogEntrySizeIsNotValid() {
+ var config = new MockLogStorageView(
+ DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE,
+ DEFAULT_SEGMENT_FILE_SIZE_BYTES,
+ 0
+ );
+
+ validate(
+ validator,
+ mock(ValidLogStorageConfiguration.class),
+ mockValidationContext(null, config),
+ "Maximum log entry size must be positive, got 0."
+ );
+ }
+
+ @Test
+ void logEntrySizeEqualToSegmentFileSizeIsNotValid() {
+ var config = new MockLogStorageView(
+ DEFAULT_MAX_CHECKPOINT_QUEUE_SIZE,
+ 10,
+ 10
+ );
+
+ validate(
+ validator,
+ mock(ValidLogStorageConfiguration.class),
+ mockValidationContext(null, config),
+ "Maximum log entry size is too big (10 bytes), maximum allowed
log entry size is 9 bytes."
+ );
+ }
+
+ private static class MockLogStorageView implements LogStorageView {
+ private final int maxCheckpointQueueSize;
+
+ private final int segmentFileSizeBytes;
+
+ private final int maxLogEntrySizeBytes;
+
+ MockLogStorageView(int maxCheckpointQueueSize, int
segmentFileSizeBytes, int maxLogEntrySizeBytes) {
+ this.maxCheckpointQueueSize = maxCheckpointQueueSize;
+ this.segmentFileSizeBytes = segmentFileSizeBytes;
+ this.maxLogEntrySizeBytes = maxLogEntrySizeBytes;
+ }
+
+ @Override
+ public int maxCheckpointQueueSize() {
+ return maxCheckpointQueueSize;
+ }
+
+ @Override
+ public long segmentFileSizeBytes() {
+ return segmentFileSizeBytes;
+ }
+
+ @Override
+ public int maxLogEntrySizeBytes() {
+ return maxLogEntrySizeBytes;
+ }
+ }
+}
diff --git
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManager.java
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManager.java
index cb7ed6a88cd..04dbb0701fa 100644
---
a/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManager.java
+++
b/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManager.java
@@ -18,6 +18,8 @@
package org.apache.ignite.internal.raft.storage.segstore;
import static java.lang.Math.toIntExact;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.UNSPECIFIED_MAX_LOG_ENTRY_SIZE;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.computeDefaultMaxLogEntrySizeBytes;
import static
org.apache.ignite.internal.raft.storage.segstore.SegmentInfo.MISSING_SEGMENT_FILE_OFFSET;
import static
org.apache.ignite.internal.raft.storage.segstore.SegmentPayload.RESET_RECORD_SIZE;
import static
org.apache.ignite.internal.raft.storage.segstore.SegmentPayload.TRUNCATE_PREFIX_RECORD_SIZE;
@@ -40,6 +42,7 @@ import
org.apache.ignite.internal.lang.IgniteInternalException;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.raft.configuration.LogStorageConfiguration;
+import org.apache.ignite.internal.raft.configuration.LogStorageView;
import
org.apache.ignite.internal.raft.storage.segstore.EntrySearchResult.SearchOutcome;
import
org.apache.ignite.internal.raft.storage.segstore.SegmentFile.WriteBuffer;
import org.apache.ignite.raft.jraft.entity.LogEntry;
@@ -118,9 +121,6 @@ class SegmentFileManager implements ManuallyCloseable {
private final Path segmentFilesDir;
- /** Configured size of a segment file. */
- private final long fileSize;
-
/** Number of stripes used by the index memtable. Should be equal to the
number of stripes in the Raft server's Disruptor. */
private final int stripes;
@@ -133,6 +133,12 @@ class SegmentFileManager implements ManuallyCloseable {
private final IndexFileManager indexFileManager;
+ /** Configured size of a segment file. */
+ private final int segmentFileSize;
+
+ /** Configured maximum log entry size. */
+ private final int maxLogEntrySize;
+
/** Lock used to block threads while a rollover is in progress. */
private final Object rolloverLock = new Object();
@@ -159,20 +165,26 @@ class SegmentFileManager implements ManuallyCloseable {
Files.createDirectories(segmentFilesDir);
- this.fileSize =
toIntExact(storageConfiguration.segmentFileSizeBytes().value());
this.stripes = stripes;
+ LogStorageView logStorageView = storageConfiguration.value();
+
+ segmentFileSize = toIntExact(logStorageView.segmentFileSizeBytes());
+
+ maxLogEntrySize = maxLogEntrySize(logStorageView);
+
indexFileManager = new IndexFileManager(baseDir);
+
checkpointer = new RaftLogCheckpointer(
nodeName,
indexFileManager,
failureProcessor,
- storageConfiguration.maxCheckpointQueueSize().value()
+ logStorageView.maxCheckpointQueueSize()
);
}
void start() throws IOException {
- LOG.info("Starting segment file manager [segmentFilesDir={},
fileSize={}].", segmentFilesDir, fileSize);
+ LOG.info("Starting segment file manager [segmentFilesDir={},
fileSize={}].", segmentFilesDir, segmentFileSize);
indexFileManager.cleanupTmpFiles();
@@ -236,7 +248,7 @@ class SegmentFileManager implements ManuallyCloseable {
private SegmentFileWithMemtable allocateNewSegmentFile(int fileOrdinal)
throws IOException {
Path path = segmentFilesDir.resolve(segmentFileName(fileOrdinal, 0));
- SegmentFile segmentFile = SegmentFile.createNew(path, fileSize);
+ SegmentFile segmentFile = SegmentFile.createNew(path, segmentFileSize);
writeHeader(segmentFile);
@@ -286,10 +298,10 @@ class SegmentFileManager implements ManuallyCloseable {
void appendEntry(long groupId, LogEntry entry, LogEntryEncoder encoder)
throws IOException {
int segmentEntrySize = SegmentPayload.size(entry, encoder);
- if (segmentEntrySize > maxPossibleEntrySize()) {
+ if (segmentEntrySize > maxLogEntrySize) {
throw new IllegalArgumentException(String.format(
"Segment entry is too big (%d bytes), maximum allowed
segment entry size: %d bytes.",
- segmentEntrySize, maxPossibleEntrySize()
+ segmentEntrySize, maxLogEntrySize
));
}
@@ -533,10 +545,6 @@ class SegmentFileManager implements ManuallyCloseable {
}
}
- private long maxPossibleEntrySize() {
- return fileSize - HEADER_RECORD.length;
- }
-
private EntrySearchResult readFromOtherSegmentFiles(long groupId, long
logIndex) throws IOException {
SegmentFilePointer segmentFilePointer =
indexFileManager.getSegmentFilePointer(groupId, logIndex);
@@ -565,4 +573,14 @@ class SegmentFileManager implements ManuallyCloseable {
return Integer.parseInt(matcher.group("ordinal"));
}
+
+ private static int maxLogEntrySize(LogStorageView storageConfiguration) {
+ int valueFromConfig = storageConfiguration.maxLogEntrySizeBytes();
+
+ if (valueFromConfig != UNSPECIFIED_MAX_LOG_ENTRY_SIZE) {
+ return valueFromConfig;
+ }
+
+ return
computeDefaultMaxLogEntrySizeBytes(storageConfiguration.segmentFileSizeBytes());
+ }
}
diff --git
a/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManagerTest.java
b/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManagerTest.java
index 39d69494f65..b5cb4f495bc 100644
---
a/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManagerTest.java
+++
b/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/segstore/SegmentFileManagerTest.java
@@ -22,6 +22,7 @@ import static java.util.concurrent.CompletableFuture.allOf;
import static java.util.concurrent.CompletableFuture.runAsync;
import static java.util.concurrent.CompletableFuture.supplyAsync;
import static java.util.stream.Collectors.toList;
+import static
org.apache.ignite.internal.raft.configuration.LogStorageConfigurationSchema.computeDefaultMaxLogEntrySizeBytes;
import static
org.apache.ignite.internal.raft.storage.segstore.ByteChannelUtils.readFully;
import static
org.apache.ignite.internal.raft.storage.segstore.SegmentFileManager.HEADER_RECORD;
import static
org.apache.ignite.internal.raft.storage.segstore.SegmentFileManager.SWITCH_SEGMENT_RECORD;
@@ -171,7 +172,7 @@ class SegmentFileManagerTest extends IgniteAbstractTest {
String expectedMessage = String.format(
"Segment entry is too big (%d bytes), maximum allowed segment
entry size: %d bytes.",
FILE_SIZE + SegmentPayload.fixedOverheadSize() + 2, // 2 bytes
for index and term.
- FILE_SIZE - HEADER_RECORD.length
+ computeDefaultMaxLogEntrySizeBytes(FILE_SIZE)
);
assertThat(e.getMessage(), is(expectedMessage));