diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 21ef983498d9..c0cad39b5059 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -20,6 +20,7 @@ import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -292,16 +293,8 @@ public class Config public int commitlog_sync_period_in_ms; public int commitlog_segment_size_in_mb = 32; - /** - * Indicates the type of compressor to use by default for flushes and sstables. - * Can be either "fast" or "adaptive". - */ - public String default_sstable_compression = System.getProperty("cassandra.default_sstable_compression", "fast"); - public ParameterizedClass commitlog_compression; - public FlushCompression flush_compression = default_sstable_compression.equals("adaptive") - ? FlushCompression.adaptive - : FlushCompression.fast; + public FlushCompression flush_compression; public int commitlog_max_compression_buffers_in_pool = 3; public Integer periodic_commitlog_sync_lag_block_in_ms; public TransparentDataEncryptionOptions transparent_data_encryption_options = new TransparentDataEncryptionOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 191cf402542a..55f52464b62d 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2081,7 +2081,9 @@ public static void setCommitLogCompression(ParameterizedClass compressor) public static Config.FlushCompression getFlushCompression() { - return conf.flush_compression; + return Objects.requireNonNullElseGet(conf.flush_compression, () -> shouldUseAdaptiveCompressionByDefault() + ? Config.FlushCompression.adaptive + : Config.FlushCompression.fast); } public static void setFlushCompression(Config.FlushCompression compression) @@ -2091,7 +2093,7 @@ public static void setFlushCompression(Config.FlushCompression compression) public static boolean shouldUseAdaptiveCompressionByDefault() { - return conf.default_sstable_compression.equals("adaptive"); + return System.getProperty("cassandra.default_sstable_compression", "fast").equals("adaptive"); } /** diff --git a/src/java/org/apache/cassandra/io/compress/AdaptiveCompressor.java b/src/java/org/apache/cassandra/io/compress/AdaptiveCompressor.java index 09064f9455b2..b4179745da2f 100644 --- a/src/java/org/apache/cassandra/io/compress/AdaptiveCompressor.java +++ b/src/java/org/apache/cassandra/io/compress/AdaptiveCompressor.java @@ -69,11 +69,10 @@ public class AdaptiveCompressor implements ICompressor { @VisibleForTesting - static final Map metrics = new EnumMap<>(Uses.class) - {{ - put(Uses.FAST_COMPRESSION, new Metrics(Uses.FAST_COMPRESSION)); - put(Uses.GENERAL, new Metrics(Uses.GENERAL)); - }}; + static final Map metrics = new EnumMap<>(Map.of( + Uses.FAST_COMPRESSION, new Metrics(Uses.FAST_COMPRESSION), + Uses.GENERAL, new Metrics(Uses.GENERAL) + )); protected static final String MIN_COMPRESSION_LEVEL_OPTION_NAME = "min_compression_level"; protected static final String MAX_COMPRESSION_LEVEL_OPTION_NAME = "max_compression_level"; diff --git a/test/unit/org/apache/cassandra/io/compress/AdaptiveCompressorTest.java b/test/unit/org/apache/cassandra/io/compress/AdaptiveCompressorTest.java index c43e0bf493aa..b8cf758e933d 100644 --- a/test/unit/org/apache/cassandra/io/compress/AdaptiveCompressorTest.java +++ b/test/unit/org/apache/cassandra/io/compress/AdaptiveCompressorTest.java @@ -55,11 +55,11 @@ public void badMaxCompactionQueueLengthParamThrowsExceptionMin() @Test public void averageRelativeTimeCompressingIsMeasuredProperly() throws IOException, InterruptedException { - var params = new AdaptiveCompressor.Params(ICompressor.Uses.GENERAL, 10, 10, 0); + var params = new AdaptiveCompressor.Params(ICompressor.Uses.GENERAL, 15, 15, 0); AdaptiveCompressor c1 = new AdaptiveCompressor(params, () -> 0.0); ByteBuffer src = getSrcByteBuffer(); ByteBuffer dest = getDstByteBuffer(c1); - for (int i = 0; i < 10000; i++) + for (int i = 0; i < 20000; i++) { compress(c1, src, dest); }