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 94dc43c21cb IGNITE-28379 Add soft storage limit configuration property
(#7885)
94dc43c21cb is described below
commit 94dc43c21cb4a7f58c3381eb3b5e0be93db3d71a
Author: Alexander Polovtcev <[email protected]>
AuthorDate: Fri Mar 27 12:15:14 2026 +0200
IGNITE-28379 Add soft storage limit configuration property (#7885)
---
.../validation/TestValidationUtil.java | 2 +-
.../LogStorageConfigurationSchema.java | 12 ++
.../LogStorageConfigurationValidator.java | 19 ++-
.../LogStorageConfigurationValidatorTest.java | 134 +++++++++------------
4 files changed, 87 insertions(+), 80 deletions(-)
diff --git
a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/validation/TestValidationUtil.java
b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/validation/TestValidationUtil.java
index d81bc72e26a..e59770680b4 100644
---
a/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/validation/TestValidationUtil.java
+++
b/modules/configuration/src/testFixtures/java/org/apache/ignite/internal/configuration/validation/TestValidationUtil.java
@@ -108,7 +108,7 @@ public class TestValidationUtil {
validator.validate(annotation, ctx);
- if (errorMessagePrefixes == null) {
+ if (errorMessagePrefixes == null || errorMessagePrefixes.length == 0) {
assertThat(argumentCaptor.getAllValues(), empty());
} else {
List<String> messages = argumentCaptor.getAllValues().stream()
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 184bdeafec0..df37ddc6fbe 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
@@ -32,6 +32,8 @@ public class LogStorageConfigurationSchema {
public static final int UNSPECIFIED_MAX_LOG_ENTRY_SIZE = -1;
+ public static final long DEFAULT_SOFT_LOG_SIZE_LIMIT_BYTES = 10L *
DEFAULT_SEGMENT_FILE_SIZE_BYTES;
+
/**
* Maximum size of the log storage checkpoint queue.
*/
@@ -52,6 +54,16 @@ public class LogStorageConfigurationSchema {
@Value(hasDefault = true)
public int maxLogEntrySizeBytes = UNSPECIFIED_MAX_LOG_ENTRY_SIZE;
+ /**
+ * Soft limit on the total size of all log storage files in bytes. When
the total size exceeds this value, the
+ * garbage collector is triggered to compact segment files and reclaim
disk space.
+ *
+ * <p>Must be at least {@link #segmentFileSizeBytes}.
+ */
+ @Value(hasDefault = true)
+ @Range(min = 1)
+ public long softLogSizeLimitBytes = DEFAULT_SOFT_LOG_SIZE_LIMIT_BYTES;
+
/**
* Computes the default maximum log entry size based on the segment file
size.
*
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
index 4e388adf5d1..8ac01466d6b 100644
---
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
@@ -31,6 +31,7 @@ class LogStorageConfigurationValidator implements
Validator<ValidLogStorageConfi
LogStorageView newValue = ctx.getNewValue();
validateLogEntrySize(newValue, ctx);
+ validateSoftLogSizeLimit(newValue, ctx);
}
private static void validateLogEntrySize(LogStorageView config,
ValidationContext<LogStorageView> ctx) {
@@ -41,7 +42,7 @@ class LogStorageConfigurationValidator implements
Validator<ValidLogStorageConfi
}
if (maxEntrySize <= 0) {
- String errorMsg = String.format("Maximum log entry size must be
positive, got %d.", maxEntrySize);
+ String errorMsg = String.format("Maximum log entry size must be
positive [maxEntrySize=%d bytes].", maxEntrySize);
ctx.addIssue(new ValidationIssue(ctx.currentKey(), errorMsg));
@@ -52,11 +53,25 @@ class LogStorageConfigurationValidator implements
Validator<ValidLogStorageConfi
if (maxEntrySize > maxAllowedEntrySize) {
String errorMsg = String.format(
- "Maximum log entry size is too big (%d bytes), maximum
allowed log entry size is %d bytes.",
+ "Maximum log entry size is too big [maxEntrySize=%d bytes,
maxAllowedEntrySize=%d bytes].",
maxEntrySize, maxAllowedEntrySize
);
ctx.addIssue(new ValidationIssue(ctx.currentKey(), errorMsg));
}
}
+
+ private static void validateSoftLogSizeLimit(LogStorageView config,
ValidationContext<LogStorageView> ctx) {
+ long softLimit = config.softLogSizeLimitBytes();
+ long segmentFileSize = config.segmentFileSizeBytes();
+
+ if (softLimit < segmentFileSize) {
+ String errorMsg = String.format(
+ "Soft log size limit must be at least the segment file
size [softLimit=%d bytes, segmentFileSize=%d bytes].",
+ softLimit, segmentFileSize
+ );
+
+ ctx.addIssue(new ValidationIssue(ctx.currentKey(), errorMsg));
+ }
+ }
}
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
index 9465d77d7ed..3a197f982f5 100644
---
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
@@ -18,106 +18,86 @@
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 static org.apache.ignite.internal.util.ArrayUtils.STRING_EMPTY_ARRAY;
+import
org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import
org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.configuration.validation.TestValidationUtil;
import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+@ExtendWith(MockitoExtension.class)
+@ExtendWith(ConfigurationExtension.class)
class LogStorageConfigurationValidatorTest extends BaseIgniteAbstractTest {
+ @Mock
+ private static ValidLogStorageConfiguration
VALID_LOG_STORAGE_CONFIGURATION;
+
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)
- );
+ void unspecifiedLogEntrySizeIsValid(
+ @InjectConfiguration(value = "mock.maxLogEntrySizeBytes=" +
UNSPECIFIED_MAX_LOG_ENTRY_SIZE, validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(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)
- );
+ void correctLogEntrySizeIsValid(
+ @InjectConfiguration(value = "mock.maxLogEntrySizeBytes=" + (int)
(DEFAULT_SEGMENT_FILE_SIZE_BYTES * 0.9), validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(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."
- );
+ void zeroLogEntrySizeIsNotValid(
+ @InjectConfiguration(value = "mock.maxLogEntrySizeBytes=0",
validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(config, "Maximum log entry size must be positive
[maxEntrySize=0 bytes].");
}
@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."
- );
+ void logEntrySizeEqualToSegmentFileSizeIsNotValid(
+ @InjectConfiguration(value = "mock { maxLogEntrySizeBytes=10,
segmentFileSizeBytes=10 }", validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(config, "Maximum log entry size is too big [maxEntrySize=10
bytes, maxAllowedEntrySize=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;
- }
+ @Test
+ void softLimitLessThanSegmentFileSizeIsNotValid(
+ @InjectConfiguration(value = "mock { softLogSizeLimitBytes=500,
segmentFileSizeBytes=1000 }", validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(config, "Soft log size limit must be at least the segment
file size [softLimit=500 bytes, segmentFileSize=1000 bytes].");
+ }
- @Override
- public int maxCheckpointQueueSize() {
- return maxCheckpointQueueSize;
- }
+ @Test
+ void softLimitEqualToSegmentFileSizeIsValid(
+ @InjectConfiguration(value = "mock { softLogSizeLimitBytes=1000,
segmentFileSizeBytes=1000 }", validate = false)
+ LogStorageConfiguration config
+ ) {
+ validate(config);
+ }
- @Override
- public long segmentFileSizeBytes() {
- return segmentFileSizeBytes;
- }
+ private void validate(LogStorageConfiguration config) {
+ validate(config, STRING_EMPTY_ARRAY);
+ }
- @Override
- public int maxLogEntrySizeBytes() {
- return maxLogEntrySizeBytes;
- }
+ private void validate(LogStorageConfiguration config, String @Nullable ...
errorMessagePrefixes) {
+ TestValidationUtil.validate(
+ validator,
+ VALID_LOG_STORAGE_CONFIGURATION,
+ mockValidationContext(null, config.value()),
+ errorMessagePrefixes
+ );
}
}