You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "Claudenw (via GitHub)" <gi...@apache.org> on 2023/03/30 09:51:15 UTC

[GitHub] [cassandra] Claudenw opened a new pull request, #2254: Cassandra 12937 sstable compression trunk

Claudenw opened a new pull request, #2254:
URL: https://github.com/apache/cassandra/pull/2254

   Same as pull request for version 4.1 except on trunk.
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176678110


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),

Review Comment:
   @Claudenw what if names of these classes are changed or we make mistake typing them out? Would not it be better if we use `LZ4Compressor.class.getName()` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182300384


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))

Review Comment:
   All deprecated stuff removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176680986


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))

Review Comment:
   this should probably go away if we remove all deprecated stuff



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#issuecomment-1503020408

   @smiklosovic Please take a look at using this as the basis for the final patch.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184990537


##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.lz4.name();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "LZ4Compressor";
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = LZ4Compressor.class.getName();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void noneTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.none.name();
+        CompressionParams params = CompressionParams.fromParameterizedClass( options );
+        // none is never enabled.
+        assertParams(params, false, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = CompressionParams.fromParameterizedClass( options );
+        // none does not set chunk length
+        assertParams(params, false, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+        options.parameters.put( CompressionParams.ENABLED,"false");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+    }
+
+    @Test
+    public void noopTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.noop.name();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "NoopCompressor";
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = NoopCompressor.class.getName();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void snappyTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.snappy.name();
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "SnappyCompressor";
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = SnappyCompressor.class.getName();
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+    }
+
+    @Test
+    public void deflateTest() {

Review Comment:
   nit: brace on newline.



##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.lz4.name();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "LZ4Compressor";
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = LZ4Compressor.class.getName();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void noneTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.none.name();
+        CompressionParams params = CompressionParams.fromParameterizedClass( options );
+        // none is never enabled.
+        assertParams(params, false, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = CompressionParams.fromParameterizedClass( options );
+        // none does not set chunk length
+        assertParams(params, false, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+        options.parameters.put( CompressionParams.ENABLED,"false");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+    }
+
+    @Test
+    public void noopTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.noop.name();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "NoopCompressor";
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = NoopCompressor.class.getName();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void snappyTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.snappy.name();
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "SnappyCompressor";
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = SnappyCompressor.class.getName();
+        paramsTest(SnappyCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+    }
+
+    @Test
+    public void deflateTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.deflate.name();
+        paramsTest(DeflateCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "DeflateCompressor";
+        paramsTest(DeflateCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = DeflateCompressor.class.getName();
+        paramsTest(DeflateCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void zstdTest() {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176692628


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;

Review Comment:
   `CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass) ? CompressorType.lz4 : null;`
   
   Here some spaces would be handy!



##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;

Review Comment:
   `StringUtils.isEmpty(sstableCompressionClass) ? CompressorType.lz4 : null;`
   
   Here some spaces would be handy!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182297393


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -55,35 +58,44 @@
     private static volatile boolean hasLoggedChunkLengthWarning;
     private static volatile boolean hasLoggedCrcCheckChanceWarning;
 
-    public static final int DEFAULT_CHUNK_LENGTH = 1024 * 16;
+    public static final int DEFAULT_CHUNK_LENGTH = 1024 * 16; // in KB
     public static final double DEFAULT_MIN_COMPRESS_RATIO = 0.0;        // Since pre-4.0 versions do not understand the
                                                                         // new compression parameter we can't use a
                                                                         // different default value.
     public static final IVersionedSerializer<CompressionParams> serializer = new Serializer();
 
     public static final String CLASS = "class";
     public static final String CHUNK_LENGTH_IN_KB = "chunk_length_in_kb";
+    /**
+     * Requires a DataStorageSpec suffix
+     */
+    public static final String CHUNK_LENGTH = "chunk_length";
+    /**
+     * Requires a DataStorageSpec suffix
+     */
+    public static final String MAX_COMPRESSED_LENGTH = "max_compressed_length";
     public static final String ENABLED = "enabled";
     public static final String MIN_COMPRESS_RATIO = "min_compress_ratio";
 
-    public static final CompressionParams DEFAULT = !CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
-                                                    ? noCompression()
-                                                    : new CompressionParams(LZ4Compressor.create(Collections.emptyMap()),
-                                                                            DEFAULT_CHUNK_LENGTH,
-                                                                            calcMaxCompressedLength(DEFAULT_CHUNK_LENGTH, DEFAULT_MIN_COMPRESS_RATIO),
-                                                                            DEFAULT_MIN_COMPRESS_RATIO,
-                                                                            Collections.emptyMap());
-
     public static final CompressionParams NOOP = new CompressionParams(NoopCompressor.create(Collections.emptyMap()),
                                                                        // 4 KiB is often the underlying disk block size
                                                                        1024 * 4,
                                                                        Integer.MAX_VALUE,
                                                                        DEFAULT_MIN_COMPRESS_RATIO,
                                                                        Collections.emptyMap());
 
+    private static final CompressionParams DEFAULT = new CompressionParams(LZ4Compressor.create(Collections.<String, String>emptyMap()),
+                                                                       DEFAULT_CHUNK_LENGTH,
+                                                                       calcMaxCompressedLength(DEFAULT_CHUNK_LENGTH, DEFAULT_MIN_COMPRESS_RATIO),
+                                                                       DEFAULT_MIN_COMPRESS_RATIO,
+                                                                       Collections.emptyMap());
+
     private static final String CRC_CHECK_CHANCE_WARNING = "The option crc_check_chance was deprecated as a compression option. " +
                                                            "You should specify it as a top-level table option instead";
 
+    @VisibleForTesting
+    static final String TOO_MANY_CHUNK_LENGTH = "Only one of 'chunk_length', 'chunk_length_kb', or 'chunk_length_in_kb' may be specified";
+
     @Deprecated public static final String SSTABLE_COMPRESSION = "sstable_compression";

Review Comment:
   I removed all the deprecated options.  However crc_check_chance is set and read by functions that are used elsewhere so the variable itself can not be removed.



##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182294538


##########
conf/cassandra.yaml:
##########
@@ -566,6 +566,48 @@ commitlog_segment_size: 32MiB
 #
 # flush_compression: fast
 
+#
+sstable_compressor:
+#   - class_name: lz4
+#     parameters:
+#       - enable: true
+#         chunk_length: 16KiB
+#         min_compress_ratio: 0.0
+#         max_comrpessed_length: 16KiB
+#         class_specific_parameter: value
+#
+#
+# The class_name is the compressor class name. It may be one of the aliases,
+# the class name of a system ICompressor implementation, or fully qualified
+# name of a class that implements ICompressor.
+#
+# class aliases are:
+#  alias   system compressor impl.
+#  lz4      LZ4Compressor,
+#  none     (null) -- disabled
+#  noop     NoopCompressor
+#  snappy   SnappyCompressor
+#  deflate  DeflateCompressor
+#  zstd     ZstdCompressor
+#
+# The standard parameters are any required or optional parameter for the instantiation of the
+# specified class, or one of the following standard parameters:
+# parameter                       usage
+# enable                Disables compression if set to false. Defaults to true.
+# chunk_length          The lenght of the compresson chunks, defaults to 16KiB
+# min_compress_ratio    The miniimal acceptable compression, must greater than or equal to 1.0.
+# max_compressed_length The maximum size for a compressed block.  Must be less than
+#                           chunk_length.   Defaults to Integer.MAX_VALUE
+# chunk_length_in_kb    The CQL compression parameter.
+#
+# Only one of the min_compress_ratio and max_compressed_length options can be specified.
+# They are  mathematically related in that
+# chunk_length / max_compressed_length = min_compress_ratio
+# if neither option is specified a min_compress_ratio of 0.0 is the default.
+#
+# Only one of chunk_length or chunk_length_in_kb may be specified.

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184986000


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -402,29 +411,47 @@ private static Integer parseChunkLength(String chLengthKB) throws ConfigurationE
      */
     private static int removeChunkLength(Map<String, String> options)
     {
-        if (options.containsKey(CHUNK_LENGTH_IN_KB))
-        {
-            if (options.containsKey(CHUNK_LENGTH_KB))
+        Integer chunk_length_in_kb = null;
+        String key = null;
+        if (options.containsKey(CHUNK_LENGTH)) {
+            key = CHUNK_LENGTH;
+            String value = options.remove(CHUNK_LENGTH);
+            try
             {
-                throw new ConfigurationException(format("The '%s' option must not be used if the chunk length is already specified by the '%s' option",
-                                                        CHUNK_LENGTH_KB,
-                                                        CHUNK_LENGTH_IN_KB));
+                chunk_length_in_kb =  new DataStorageSpec.IntKibibytesBound(value).toKibibytes();
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue(CHUNK_LENGTH, e.getMessage(), value));
             }
-
-            return parseChunkLength(options.remove(CHUNK_LENGTH_IN_KB));
         }
 
-        if (options.containsKey(CHUNK_LENGTH_KB))
+        if (options.containsKey(CHUNK_LENGTH_IN_KB))
         {
-            if (!hasLoggedChunkLengthWarning)
+            key = CHUNK_LENGTH_IN_KB;
+            if (chunk_length_in_kb != null)
+                throw new ConfigurationException(TOO_MANY_CHUNK_LENGTH);
+            else
             {
-                hasLoggedChunkLengthWarning = true;
-                logger.warn("The {} option has been deprecated. You should use {} instead",
-                                   CHUNK_LENGTH_KB,
-                                   CHUNK_LENGTH_IN_KB);
+                String chLengthKB = options.remove(CHUNK_LENGTH_IN_KB);
+                try
+                {
+                    int parsed = Integer.parseInt(chLengthKB);
+                    if (parsed > Integer.MAX_VALUE / 1024)
+                        throw new ConfigurationException(invalidValue(CHUNK_LENGTH_IN_KB, "Value is too large", parsed));
+                    if (parsed <= 0)
+                        throw new ConfigurationException(invalidValue(CHUNK_LENGTH_IN_KB, "May not be <= 0", parsed));
+                    chunk_length_in_kb = 1024 * parsed;
+                }
+                catch (NumberFormatException e)
+                {
+                    throw new ConfigurationException(invalidValue(CHUNK_LENGTH_IN_KB, e.getMessage(), chLengthKB));
+                }
             }
+        }
 
-            return parseChunkLength(options.remove(CHUNK_LENGTH_KB));
+        if (chunk_length_in_kb != null) {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184986922


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -518,23 +527,36 @@ private static boolean removeEnabled(Map<String, String> options)
         return enabled == null || Boolean.parseBoolean(enabled);
     }
 
-    // chunkLength must be a power of 2 because we assume so when
-    // computing the chunk number from an uncompressed file offset (see
-    // CompressedRandomAccessReader.decompresseChunk())
-    public void validate() throws ConfigurationException
-    {
-        // if chunk length was not set (chunkLength == null), this is fine, default will be used
+    private static void validateChunkLength(String key, int chunkLength) throws ConfigurationException{
         if (chunkLength <= 0)
-            throw new ConfigurationException("Invalid negative or null " + CHUNK_LENGTH_IN_KB);
+            throw new ConfigurationException(invalidValue(key, "May not be <= 0", chunkLength));
 
+        // chunkLength must be a power of 2 because we assume so when
+        // computing the chunk number from an uncompressed file offset (see
+        // CompressedRandomAccessReader.decompresseChunk())
         if ((chunkLength & (chunkLength - 1)) != 0)
-            throw new ConfigurationException(CHUNK_LENGTH_IN_KB + " must be a power of 2");
+            throw new ConfigurationException(invalidValue(key,  "Must be a power of 2", chunkLength));
+    }
 
+    private static void validateMinCompressRatio(double ratio) throws ConfigurationException {
+        if (ratio != DEFAULT_MIN_COMPRESS_RATIO && ratio < 1.0)
+            throw new ConfigurationException(invalidValue(MIN_COMPRESS_RATIO , "Can either be 0 or greater than or equal to 1", ratio));
+
+    }
+
+    private static void validateMaxCompressedLength(int maxCompressedLength, int chunkLength) throws ConfigurationException {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176688094


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {

Review Comment:
   I do not understand why it is not done just like this:
   
   ````
           boolean enabled = removeEnabled(options);
   ````



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176678926


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {

Review Comment:
   please take care of braces on new lines in this whole enum.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176661551


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {

Review Comment:
   We could quickly return null if `name` is null so we are not looping unnecessarily. Please keep the braces on the new line in this whole method as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176661551


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {

Review Comment:
   We could quickly return null if `name` is null so we are not looping unnecessarily. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176659483


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -191,6 +191,9 @@
 
     private static Map<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories;
 
+    /** The sstable compression options */

Review Comment:
   The usefulness of this comment is questionable. It is pretty obvious this is about SSTable compression from the name of that variable itself. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#issuecomment-1506847872

   @smiklosovic Please review this pull request.  I think it is complete and correct.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182301073


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {

Review Comment:
   holdover from old code.  Fixed now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184989675


##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {

Review Comment:
   nit: brace on newline.



##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184986527


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -518,23 +527,36 @@ private static boolean removeEnabled(Map<String, String> options)
         return enabled == null || Boolean.parseBoolean(enabled);
     }
 
-    // chunkLength must be a power of 2 because we assume so when
-    // computing the chunk number from an uncompressed file offset (see
-    // CompressedRandomAccessReader.decompresseChunk())
-    public void validate() throws ConfigurationException
-    {
-        // if chunk length was not set (chunkLength == null), this is fine, default will be used
+    private static void validateChunkLength(String key, int chunkLength) throws ConfigurationException{

Review Comment:
   nit: brace on newline.



##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -518,23 +527,36 @@ private static boolean removeEnabled(Map<String, String> options)
         return enabled == null || Boolean.parseBoolean(enabled);
     }
 
-    // chunkLength must be a power of 2 because we assume so when
-    // computing the chunk number from an uncompressed file offset (see
-    // CompressedRandomAccessReader.decompresseChunk())
-    public void validate() throws ConfigurationException
-    {
-        // if chunk length was not set (chunkLength == null), this is fine, default will be used
+    private static void validateChunkLength(String key, int chunkLength) throws ConfigurationException{
         if (chunkLength <= 0)
-            throw new ConfigurationException("Invalid negative or null " + CHUNK_LENGTH_IN_KB);
+            throw new ConfigurationException(invalidValue(key, "May not be <= 0", chunkLength));
 
+        // chunkLength must be a power of 2 because we assume so when
+        // computing the chunk number from an uncompressed file offset (see
+        // CompressedRandomAccessReader.decompresseChunk())
         if ((chunkLength & (chunkLength - 1)) != 0)
-            throw new ConfigurationException(CHUNK_LENGTH_IN_KB + " must be a power of 2");
+            throw new ConfigurationException(invalidValue(key,  "Must be a power of 2", chunkLength));
+    }
 
+    private static void validateMinCompressRatio(double ratio) throws ConfigurationException {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176678488


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;

Review Comment:
   this might be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176688756


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -270,6 +339,10 @@ public int chunkLength()
         return chunkLength;
     }
 
+    double minCompressRatio() {

Review Comment:
   nit: braces on new line



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176739561


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)

Review Comment:
   I wonder why we need this entire `switch` at all. If we treat all compressors as equal, why do we need to have a switch and then a special `else` branch? Why not to instantiate all compressors the same way as it is done for case of a custom one? I just do not like that we are making the difference between in-built compressors and custom ones. A compressor is just a compressor. The fact that it is accidentally in-built is irrelevant. 
   
   It will be also aligned with how we are doing it in cassandra.yaml, conceptually. If we are making a flat map and we treat all compressors equal, irrelevant where they are from, we would just apply the same reasoning there and it would simplify the code as well. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176691308


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));

Review Comment:
   here too etc ... more cases like this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176710113


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)
+            {
+                case none:
+                    cp = new CompressionParams((ICompressor) null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case lz4:
+                    cp = new CompressionParams(enabled?LZ4Compressor.create(options):null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case snappy:
+                    cp = new CompressionParams(enabled?SnappyCompressor.instance:null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case deflate:
+                    cp = new CompressionParams(enabled?DeflateCompressor.instance:null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case zstd:
+                    cp = new CompressionParams(enabled?ZstdCompressor.create(options):null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case noop:
+                    cp = new CompressionParams(enabled?NoopCompressor.create(options):null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+            }
+        } else {
+            cp = new CompressionParams(enabled?sstableCompressionClass:null, options, chunk_length_in_kb, min_compress_ratio);
+            if (enabled && cp.getSstableCompressor() == null)

Review Comment:
   This might go away when deprecated stuff goes away as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182299156


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182295155


##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -191,6 +191,9 @@
 
     private static Map<String, Supplier<SSTableFormat<?, ?>>> sstableFormatFactories;
 
+    /** The sstable compression options */

Review Comment:
   removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182301962


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -402,29 +475,55 @@ private static Integer parseChunkLength(String chLengthKB) throws ConfigurationE
      */
     private static int removeChunkLength(Map<String, String> options)

Review Comment:
   I believe that I have fixed the braces and space changes requested through out this review.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


Re: [PR] Cassandra 12937 sstable compression rebased to trunk [cassandra]

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw closed pull request #2254: Cassandra 12937 sstable compression rebased to  trunk
URL: https://github.com/apache/cassandra/pull/2254


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


Re: [PR] Cassandra 12937 sstable compression rebased to trunk [cassandra]

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#issuecomment-2006270783

   No longer current


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176671453


##########
test/conf/cassandra-with-sstable-compressor.yaml:
##########
@@ -0,0 +1,110 @@
+
+cluster_name: Test Cluster
+# memtable_allocation_type: heap_buffers
+memtable_allocation_type: offheap_objects
+commitlog_sync: batch
+commitlog_sync_batch_window_in_ms: 1.0
+commitlog_segment_size: 5MiB
+commitlog_directory: build/test/cassandra/commitlog
+# commitlog_compression:
+# - class_name: LZ4Compressor
+cdc_raw_directory: build/test/cassandra/cdc_raw
+cdc_enabled: false
+hints_directory: build/test/cassandra/hints
+partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
+listen_address: 127.0.0.1
+storage_port: 7012
+ssl_storage_port: 17012
+start_native_transport: true
+native_transport_port: 9042
+column_index_size: 4KiB
+saved_caches_directory: build/test/cassandra/saved_caches
+data_file_directories:
+    - build/test/cassandra/data
+disk_access_mode: mmap
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1:7012"
+endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
+dynamic_snitch: true
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+incremental_backups: true
+concurrent_compactors: 4
+compaction_throughput: 0MiB/s
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size: 16MiB
+user_defined_functions_enabled: true
+scripted_user_defined_functions_enabled: true
+prepared_statements_cache_size: 1MiB
+corrupted_tombstone_strategy: exception
+stream_entire_sstables: true
+stream_throughput_outbound: 23841858MiB/s
+sasi_indexes_enabled: true
+materialized_views_enabled: true
+drop_compact_storage_enabled: true
+file_cache_enabled: true
+auto_hints_cleanup_enabled: true
+
+read_thresholds_enabled: true
+coordinator_read_size_warn_threshold: 1024KiB
+coordinator_read_size_fail_threshold: 4096KiB
+local_read_size_warn_threshold: 4096KiB
+local_read_size_fail_threshold: 8192KiB
+row_index_read_size_warn_threshold: 4096KiB
+row_index_read_size_fail_threshold: 8192KiB
+
+memtable:
+    configurations:
+        skiplist:
+            inherits: default
+            class_name: SkipListMemtable
+        skiplist_sharded:
+            class_name: ShardedSkipListMemtable
+            parameters:
+                serialize_writes: false
+                shards: 4
+        skiplist_sharded_locking:
+            inherits: skiplist_sharded
+            parameters:
+                serialize_writes: true
+        skiplist_remapped:
+            inherits: skiplist
+        test_fullname:
+            inherits: default
+            class_name: org.apache.cassandra.db.memtable.TestMemtable
+        test_shortname:
+            class_name: TestMemtable
+            parameters:
+                skiplist: true  # note: YAML must interpret this as string, not a boolean
+        test_empty_class:
+            class_name: ""
+        test_missing_class:
+            parameters:
+        test_unknown_class:
+            class_name: NotExisting
+        test_invalid_param:
+            class_name: SkipListMemtable
+            parameters:
+                invalid: throw
+        test_invalid_extra_param:
+            inherits: test_shortname
+            parameters:
+                invalid: throw
+        test_invalid_factory_method:
+            class_name: org.apache.cassandra.cql3.validation.operations.CreateTest$InvalidMemtableFactoryMethod
+        test_invalid_factory_field:
+            class_name: org.apache.cassandra.cql3.validation.operations.CreateTest$InvalidMemtableFactoryField
+
+sstable_compressor:

Review Comment:
   Please, dont forget to rename this as well to `sstable_compression`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176676449


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -55,35 +58,44 @@
     private static volatile boolean hasLoggedChunkLengthWarning;
     private static volatile boolean hasLoggedCrcCheckChanceWarning;
 
-    public static final int DEFAULT_CHUNK_LENGTH = 1024 * 16;
+    public static final int DEFAULT_CHUNK_LENGTH = 1024 * 16; // in KB
     public static final double DEFAULT_MIN_COMPRESS_RATIO = 0.0;        // Since pre-4.0 versions do not understand the
                                                                         // new compression parameter we can't use a
                                                                         // different default value.
     public static final IVersionedSerializer<CompressionParams> serializer = new Serializer();
 
     public static final String CLASS = "class";
     public static final String CHUNK_LENGTH_IN_KB = "chunk_length_in_kb";
+    /**
+     * Requires a DataStorageSpec suffix
+     */
+    public static final String CHUNK_LENGTH = "chunk_length";
+    /**
+     * Requires a DataStorageSpec suffix
+     */
+    public static final String MAX_COMPRESSED_LENGTH = "max_compressed_length";
     public static final String ENABLED = "enabled";
     public static final String MIN_COMPRESS_RATIO = "min_compress_ratio";
 
-    public static final CompressionParams DEFAULT = !CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
-                                                    ? noCompression()
-                                                    : new CompressionParams(LZ4Compressor.create(Collections.emptyMap()),
-                                                                            DEFAULT_CHUNK_LENGTH,
-                                                                            calcMaxCompressedLength(DEFAULT_CHUNK_LENGTH, DEFAULT_MIN_COMPRESS_RATIO),
-                                                                            DEFAULT_MIN_COMPRESS_RATIO,
-                                                                            Collections.emptyMap());
-
     public static final CompressionParams NOOP = new CompressionParams(NoopCompressor.create(Collections.emptyMap()),
                                                                        // 4 KiB is often the underlying disk block size
                                                                        1024 * 4,
                                                                        Integer.MAX_VALUE,
                                                                        DEFAULT_MIN_COMPRESS_RATIO,
                                                                        Collections.emptyMap());
 
+    private static final CompressionParams DEFAULT = new CompressionParams(LZ4Compressor.create(Collections.<String, String>emptyMap()),
+                                                                       DEFAULT_CHUNK_LENGTH,
+                                                                       calcMaxCompressedLength(DEFAULT_CHUNK_LENGTH, DEFAULT_MIN_COMPRESS_RATIO),
+                                                                       DEFAULT_MIN_COMPRESS_RATIO,
+                                                                       Collections.emptyMap());
+
     private static final String CRC_CHECK_CHANCE_WARNING = "The option crc_check_chance was deprecated as a compression option. " +
                                                            "You should specify it as a top-level table option instead";
 
+    @VisibleForTesting
+    static final String TOO_MANY_CHUNK_LENGTH = "Only one of 'chunk_length', 'chunk_length_kb', or 'chunk_length_in_kb' may be specified";
+
     @Deprecated public static final String SSTABLE_COMPRESSION = "sstable_compression";

Review Comment:
   @Claudenw I would remove all these deprecated options as part of this PR. At least `crc_check_chance` seems to be deprecated in 3.0 so it is eligible for deletion. I think that the rest is good to remove too but we would need to confirm this as git history is little bit unclear here.
   
   cc @michaelsembwever 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176679835


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {

Review Comment:
   braces around one return statement are not necessary, are they?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176739561


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)

Review Comment:
   I wonder why we need this entire `switch` at all. If we treat all compressors as equal, why do we need to have a switch and then a special `else` branch? Why not to instantiate all compressors the same way as it is done for case of a custom one? I just do not like that we are making the difference between in-built compressors and custom ones. A compressor is just a compressor. The fact that it is accidentally in-built is irrelevant. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1175529411


##########
conf/cassandra.yaml:
##########
@@ -566,6 +566,48 @@ commitlog_segment_size: 32MiB
 #
 # flush_compression: fast
 
+#
+sstable_compressor:
+#   - class_name: lz4
+#     parameters:
+#       - enable: true
+#         chunk_length: 16KiB
+#         min_compress_ratio: 0.0
+#         max_comrpessed_length: 16KiB
+#         class_specific_parameter: value
+#
+#
+# The class_name is the compressor class name. It may be one of the aliases,
+# the class name of a system ICompressor implementation, or fully qualified
+# name of a class that implements ICompressor.
+#
+# class aliases are:
+#  alias   system compressor impl.
+#  lz4      LZ4Compressor,
+#  none     (null) -- disabled
+#  noop     NoopCompressor
+#  snappy   SnappyCompressor
+#  deflate  DeflateCompressor
+#  zstd     ZstdCompressor
+#
+# The standard parameters are any required or optional parameter for the instantiation of the
+# specified class, or one of the following standard parameters:
+# parameter                       usage
+# enable                Disables compression if set to false. Defaults to true.
+# chunk_length          The lenght of the compresson chunks, defaults to 16KiB
+# min_compress_ratio    The miniimal acceptable compression, must greater than or equal to 1.0.
+# max_compressed_length The maximum size for a compressed block.  Must be less than
+#                           chunk_length.   Defaults to Integer.MAX_VALUE
+# chunk_length_in_kb    The CQL compression parameter.
+#
+# Only one of the min_compress_ratio and max_compressed_length options can be specified.
+# They are  mathematically related in that
+# chunk_length / max_compressed_length = min_compress_ratio
+# if neither option is specified a min_compress_ratio of 0.0 is the default.
+#
+# Only one of chunk_length or chunk_length_in_kb may be specified.

Review Comment:
   nit: my bad, please put the comment section before the commented out config. this is the norm in this file.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176655715


##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -1087,6 +1087,9 @@ public static PaxosStatePurging fromBoolean(boolean enabled)
     public volatile long min_tracked_partition_tombstone_count = 5000;
     public volatile boolean top_partitions_enabled = true;
 
+    @Nullable
+    public ParameterizedClass sstable_compressor;

Review Comment:
   sstable_compressor -> sstable_compression



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182296122


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {

Review Comment:
   fixed.  Also reorganized the Enum order so that `none` is last since we are capturing it first.



##########
test/conf/cassandra-with-sstable-compressor.yaml:
##########
@@ -0,0 +1,110 @@
+
+cluster_name: Test Cluster
+# memtable_allocation_type: heap_buffers
+memtable_allocation_type: offheap_objects
+commitlog_sync: batch
+commitlog_sync_batch_window_in_ms: 1.0
+commitlog_segment_size: 5MiB
+commitlog_directory: build/test/cassandra/commitlog
+# commitlog_compression:
+# - class_name: LZ4Compressor
+cdc_raw_directory: build/test/cassandra/cdc_raw
+cdc_enabled: false
+hints_directory: build/test/cassandra/hints
+partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
+listen_address: 127.0.0.1
+storage_port: 7012
+ssl_storage_port: 17012
+start_native_transport: true
+native_transport_port: 9042
+column_index_size: 4KiB
+saved_caches_directory: build/test/cassandra/saved_caches
+data_file_directories:
+    - build/test/cassandra/data
+disk_access_mode: mmap
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1:7012"
+endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
+dynamic_snitch: true
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+incremental_backups: true
+concurrent_compactors: 4
+compaction_throughput: 0MiB/s
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size: 16MiB
+user_defined_functions_enabled: true
+scripted_user_defined_functions_enabled: true
+prepared_statements_cache_size: 1MiB
+corrupted_tombstone_strategy: exception
+stream_entire_sstables: true
+stream_throughput_outbound: 23841858MiB/s
+sasi_indexes_enabled: true
+materialized_views_enabled: true
+drop_compact_storage_enabled: true
+file_cache_enabled: true
+auto_hints_cleanup_enabled: true
+
+read_thresholds_enabled: true
+coordinator_read_size_warn_threshold: 1024KiB
+coordinator_read_size_fail_threshold: 4096KiB
+local_read_size_warn_threshold: 4096KiB
+local_read_size_fail_threshold: 8192KiB
+row_index_read_size_warn_threshold: 4096KiB
+row_index_read_size_fail_threshold: 8192KiB
+
+memtable:
+    configurations:
+        skiplist:
+            inherits: default
+            class_name: SkipListMemtable
+        skiplist_sharded:
+            class_name: ShardedSkipListMemtable
+            parameters:
+                serialize_writes: false
+                shards: 4
+        skiplist_sharded_locking:
+            inherits: skiplist_sharded
+            parameters:
+                serialize_writes: true
+        skiplist_remapped:
+            inherits: skiplist
+        test_fullname:
+            inherits: default
+            class_name: org.apache.cassandra.db.memtable.TestMemtable
+        test_shortname:
+            class_name: TestMemtable
+            parameters:
+                skiplist: true  # note: YAML must interpret this as string, not a boolean
+        test_empty_class:
+            class_name: ""
+        test_missing_class:
+            parameters:
+        test_unknown_class:
+            class_name: NotExisting
+        test_invalid_param:
+            class_name: SkipListMemtable
+            parameters:
+                invalid: throw
+        test_invalid_extra_param:
+            inherits: test_shortname
+            parameters:
+                invalid: throw
+        test_invalid_factory_method:
+            class_name: org.apache.cassandra.cql3.validation.operations.CreateTest$InvalidMemtableFactoryMethod
+        test_invalid_factory_field:
+            class_name: org.apache.cassandra.cql3.validation.operations.CreateTest$InvalidMemtableFactoryField
+
+sstable_compressor:

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184985094


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -402,29 +411,47 @@ private static Integer parseChunkLength(String chLengthKB) throws ConfigurationE
      */
     private static int removeChunkLength(Map<String, String> options)
     {
-        if (options.containsKey(CHUNK_LENGTH_IN_KB))
-        {
-            if (options.containsKey(CHUNK_LENGTH_KB))
+        Integer chunk_length_in_kb = null;
+        String key = null;
+        if (options.containsKey(CHUNK_LENGTH)) {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176691030


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);

Review Comment:
   please remove these spaces before / after argument in a method. There are various instances of that in this method in general.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176697583


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -402,29 +475,55 @@ private static Integer parseChunkLength(String chLengthKB) throws ConfigurationE
      */
     private static int removeChunkLength(Map<String, String> options)

Review Comment:
   this whole method could use some love on formatting, spaces, braces, no braces if one statement only ... 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182302744


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)
+            {
+                case none:
+                    cp = new CompressionParams((ICompressor) null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case lz4:
+                    cp = new CompressionParams(enabled?LZ4Compressor.create(options):null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case snappy:
+                    cp = new CompressionParams(enabled?SnappyCompressor.instance:null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);

Review Comment:
   The test factory does it this way, but I have reverted to create method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182294907


##########
conf/cassandra.yaml:
##########
@@ -566,6 +566,48 @@ commitlog_segment_size: 32MiB
 #
 # flush_compression: fast
 
+#
+sstable_compressor:

Review Comment:
   fixed



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -1087,6 +1087,9 @@ public static PaxosStatePurging fromBoolean(boolean enabled)
     public volatile long min_tracked_partition_tombstone_count = 5000;
     public volatile boolean top_partitions_enabled = true;
 
+    @Nullable
+    public ParameterizedClass sstable_compressor;

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184987842


##########
test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java:
##########
@@ -33,6 +33,10 @@
 import java.util.stream.Stream;
 
 import com.google.common.collect.Sets;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.util.File;

Review Comment:
   nit: wrong location



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184990245


##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.lz4.name();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "LZ4Compressor";
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = LZ4Compressor.class.getName();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void noneTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.none.name();
+        CompressionParams params = CompressionParams.fromParameterizedClass( options );
+        // none is never enabled.
+        assertParams(params, false, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = CompressionParams.fromParameterizedClass( options );
+        // none does not set chunk length
+        assertParams(params, false, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+        options.parameters.put( CompressionParams.ENABLED,"false");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+    }
+
+    @Test
+    public void noopTest() {

Review Comment:
   nit: brace on newline.



##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.lz4.name();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "LZ4Compressor";
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = LZ4Compressor.class.getName();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void noneTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.none.name();
+        CompressionParams params = CompressionParams.fromParameterizedClass( options );
+        // none is never enabled.
+        assertParams(params, false, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = CompressionParams.fromParameterizedClass( options );
+        // none does not set chunk length
+        assertParams(params, false, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, null);
+
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+        options.parameters.put( CompressionParams.ENABLED,"false");
+        params = CompressionParams.fromParameterizedClass( options );
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+
+    }
+
+    @Test
+    public void noopTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.noop.name();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "NoopCompressor";
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = NoopCompressor.class.getName();
+        paramsTest(NoopCompressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void snappyTest() {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184989965


##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {

Review Comment:
   nit: brace on newline.



##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map
+        ParameterizedClass options = new ParameterizedClass();
+        CompressionParams params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options = emptyParameterizedClass();
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().isEmpty()).isTrue();
+
+        options.parameters.put( "foo", "bar");
+        params = CompressionParams.fromParameterizedClass(options);
+        params = CompressionParams.fromParameterizedClass(options);
+        assertThat( params.getOtherOptions()).isNotNull();
+        assertThat( params.getOtherOptions().get("foo")).isEqualTo("bar");
+    }
+
+    // Tests chunklength settings for both Options and Map.
+    private static <T> void chunkLengthTest(BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        // CHUNK_LENGTH
+
+        // test empty string
+        put.accept(CompressionParams.CHUNK_LENGTH, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // text zero string
+        put.accept(CompressionParams.CHUNK_LENGTH, "0MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+
+        // test properly formated value
+        put.accept(CompressionParams.CHUNK_LENGTH, "1MiB");
+        CompressionParams params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        // test bad string
+        put.accept(CompressionParams.CHUNK_LENGTH, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value");
+
+        // test not power of 2
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH);
+
+
+        // CHUNK_LENGTH_IN_KB
+        // same tests as above
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "1");
+        params = func.apply(instance);
+        assertEquals(1024, params.chunkLength());
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value");
+
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("Must be a power of 2");
+
+        // test negative value
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "-1");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'chunk_length_in_kb' value")
+                                                               .withMessageContaining("May not be <= 0");
+
+        remove.accept(CompressionParams.CHUNK_LENGTH_IN_KB);
+
+
+
+
+        // TEST COMBINATIONS
+        put.accept(CompressionParams.CHUNK_LENGTH, "3MiB");
+        put.accept(CompressionParams.CHUNK_LENGTH_IN_KB, "2");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessage(CompressionParams.TOO_MANY_CHUNK_LENGTH);
+    }
+
+    @Test
+    public void chunkLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        chunkLengthTest( options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        Consumer<String> remove = (s) -> map.remove(s);
+        chunkLengthTest( map::put,remove, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void minCompressRatioTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+
+        CompressionParams params = func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.0" ); //CompressionParams.DEFAULT_MIN_COMPRESS_RATIO
+        params =func.apply(instance);
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "0.3");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO, "1.3");
+        params = func.apply(instance);
+        assertEquals(1.3, params.minCompressRatio(), Double.MIN_VALUE);
+        assertEquals( (int) Math.ceil(CompressionParams.DEFAULT_CHUNK_LENGTH / 1.3), params.maxCompressedLength());
+
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,  "-1.0");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining( "Invalid 'min_compress_ratio' value")
+                                                               .withMessageContaining("Can either be 0 or greater than or equal to 1");
+    }
+
+    @Test
+    public void minCompressRatioTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        minCompressRatioTest( options.parameters::put, CompressionParams::fromParameterizedClass, options );
+
+        Map<String,String> map = new HashMap<String,String>();
+        map.put( CompressionParams.CLASS, "lz4");
+        minCompressRatioTest( map::put, CompressionParams::fromMap, map );
+    }
+
+    private static <T> void maxCompressedLengthTest(BiConsumer<String,String> put,  Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"");
+        params = func.apply(instance);
+        assertEquals(Integer.MAX_VALUE, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertEquals(4*1024, params.maxCompressedLength());
+        assertEquals(CompressionParams.DEFAULT_CHUNK_LENGTH / (4.0 * 1024), params.minCompressRatio(), Double.MIN_VALUE);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"badvalue");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> func.apply(instance))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value")
+                                                               .withMessageContaining("Invalid data storage");
+    }
+
+    @Test
+    public void maxCompressedLengthTest()
+    {
+        ParameterizedClass options = emptyParameterizedClass();
+        maxCompressedLengthTest(options.parameters::put, CompressionParams::fromParameterizedClass, options);
+
+        Map<String, String> map = new HashMap<String, String>();
+        map.put(CompressionParams.CLASS, "lz4");
+        maxCompressedLengthTest(map::put, CompressionParams::fromMap, map);
+    }
+
+    @Test
+    public void maxCompressionLengthAndMinCompressRatioTest() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.parameters.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        options.parameters.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromParameterizedClass(options))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+
+        Map<String,String> map = new HashMap<>();
+        map.put( CompressionParams.CLASS, "lz4");
+        map.put( CompressionParams.MIN_COMPRESS_RATIO, "1.0");
+        map.put( CompressionParams.MAX_COMPRESSED_LENGTH, "4Gib");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> CompressionParams.fromMap(map))
+                                                               .withMessage("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+    }
+
+    private static void assertParams(CompressionParams params, boolean enabled, int chunkLength, int maxCompressedLength, double minCompressRatio, Class<?> compressor)
+    {
+        assertThat(params.isEnabled()).isEqualTo(enabled);
+        assertThat(params.chunkLength()).isEqualTo(chunkLength);
+        assertThat(params.maxCompressedLength()).isEqualTo(maxCompressedLength);
+        assertThat(params.minCompressRatio()).isEqualTo(minCompressRatio);
+        if (compressor != null)
+        {
+            assertThat(params.getSstableCompressor()).isInstanceOf(compressor);
+        } else
+        {
+            assertThat(params.getSstableCompressor()).isNull();
+        }
+    }
+
+
+    @Test
+    public void defaultTest()
+    {
+        CompressionParams params = CompressionParams.fromParameterizedClass( emptyParameterizedClass() );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromParameterizedClass( null );
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+
+        params = CompressionParams.fromMap(Collections.EMPTY_MAP );
+        assertParams(params,true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, LZ4Compressor.class);
+        roundTripMapTest(params);
+    }
+
+    private static <T> void paramsTest(Class<?> clazz, BiConsumer<String,String> put, Consumer<String> remove, Function<T,CompressionParams> func, T instance)
+    {
+        CompressionParams params = func.apply(instance);
+        assertParams(params, true, CompressionParams.DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.CHUNK_LENGTH,"4MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, Integer.MAX_VALUE, CompressionParams.DEFAULT_MIN_COMPRESS_RATIO, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2MiB");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.MAX_COMPRESSED_LENGTH,"2097151KiB");
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() ->func.apply(instance))
+        .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+        assertParams(params, true, 4*1024, 2*1024, 2.0, clazz);
+        roundTripMapTest(params);
+
+        remove.accept(CompressionParams.MAX_COMPRESSED_LENGTH);
+        put.accept( CompressionParams.MIN_COMPRESS_RATIO,"1.5");
+        params = func.apply(instance);
+        assertParams(params, true, 4*1024, 2731, 1.5, clazz);
+        roundTripMapTest(params);
+
+        put.accept( CompressionParams.ENABLED,"false");
+        params = func.apply(instance);
+        assertParams(params, false, 4*1024, 2731, 1.5, null);
+        // round tripped disabled params return disabled default constructor version
+    }
+
+
+    @Test
+    public void constructorTest() {
+        Map<String,String> map = new HashMap<>();
+
+        // chunk length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+        .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // chunk length = 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  -1, 0.0))
+                                                               .withMessage("Invalid 'chunk_length' value for the 'compression' option.  May not be <= 0: -1");
+
+        // min compress ratio < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, -1.0))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // 0 < min compress ratio < 1
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(), map,  CompressionParams.DEFAULT_CHUNK_LENGTH, 0.5))
+                                                               .withMessageContaining("Invalid 'min_compress_ratio' value for the 'compression' option.  Can either be 0 or greater than or equal to 1");
+
+        // max compressed length > chunk length
+        int len = 1 << 30;
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  len, len+1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option: Must be less than or equal to chunk length");
+
+        // max compressed length < 0
+        assertThatExceptionOfType(ConfigurationException.class).isThrownBy(() -> new CompressionParams( TestCompressor.class.getName(),  CompressionParams.DEFAULT_CHUNK_LENGTH, -1, map ))
+                                                               .withMessageContaining("Invalid 'max_compressed_length' value for the 'compression' option.  May not be less than zero: -1");
+    }
+
+    private static void roundTripMapTest(CompressionParams params) {
+        CompressionParams other = CompressionParams.fromMap( params.asMap() );
+        assertThat( params.getOtherOptions() ).isEqualTo( other.getOtherOptions() );
+        assertThat( params.maxCompressedLength()).isEqualTo( other.maxCompressedLength());
+        assertThat( params.minCompressRatio()).isEqualTo( other.minCompressRatio());
+        assertThat(params.chunkLength()).isEqualTo( other.chunkLength() );
+        assertThat(params.isEnabled()).isEqualTo( other.isEnabled());
+        assertThat(params.getCrcCheckChance()).isEqualTo( other.getCrcCheckChance());
+        assertThat(params.klass()).isEqualTo( other.klass());
+    }
+
+    @Test
+    public void lz4Test() {
+        ParameterizedClass options = emptyParameterizedClass();
+        options.class_name = CompressionParams.CompressorType.lz4.name();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = "LZ4Compressor";
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+
+        options.parameters.clear();
+        options.class_name = LZ4Compressor.class.getName();
+        paramsTest(LZ4Compressor.class, options.parameters::put, options.parameters::remove, CompressionParams::fromParameterizedClass, options );
+    }
+
+    @Test
+    public void noneTest() {

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1169617046


##########
conf/cassandra.yaml:
##########
@@ -566,6 +566,35 @@ commitlog_segment_size: 32MiB
 #
 # flush_compression: fast
 
+
+# Defines the compression params used to read/write the sstables.  If undefined the default is LZ4 with 16kb chunk
+# length and min_compress_ratio of 0.0 (see CompressionParams.DEFAULT_MIN_COMPRESS_RATIO).
+# To disable compression set `type: none` or set the `enable` parameter as `false`.
+# This is the default options for sstable compression, the (schema) table-level settings will override this option.
+#sstable_compressor:
+#
+  #enable=false # defaults to true
+#
+  #chunk_length: 16KiB : defaults to 16KiB
+#
+  # min_compress_ratio: 0.0
+  # max_compressed_length: 16KiB

Review Comment:
   For readability, make the example standalone, add comments afterwards.
   ```suggestion
   #
   # sstable_compressor:
     # enable=true 
     # chunk_length: 16KiB
     # min_compress_ratio: 0.0
     # max_compressed_length: 16KiB
     # - class_name:
     # parameters:
        # - param1 : value
   #     
   # 
   # enable defaults to true 
   # chunk_length defaults to 16KiB
   # Only one of the min_compress_ratio and max_compressed_length options can be specified.  
   # The mathematically related in that
   # chunk_length / max_compressed_length = min_compress_ratio
   # if neither option is specified a min_compress_ratio of 0.0 is the default.
   # A min_compress_ratio <= 0 implies max_compressed_length of Integer.MAX_VALUE KiB
   # The class_name is the compressor class name. It may be one of the following aliases:
   # lz4,  none,  noop,  snappy,  deflate,  or zstd; or it the fully qualified  name of a class
   # that implements ICompressor
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176700271


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)
+            {
+                case none:
+                    cp = new CompressionParams((ICompressor) null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);

Review Comment:
   IDEA tells me that you do not need to cast `null` to `ICompressor`. It can be just `null`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176705606


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)
+            {
+                case none:
+                    cp = new CompressionParams((ICompressor) null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case lz4:
+                    cp = new CompressionParams(enabled?LZ4Compressor.create(options):null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);
+                    break;
+                case snappy:
+                    cp = new CompressionParams(enabled?SnappyCompressor.instance:null, chunk_length_in_kb, max_compressed_length, min_compress_ratio, options);

Review Comment:
   shouldnt be here `SnappyCompressor.create(options)` instead of `SnappyCompressor.instance`? Same for DeflateCompressor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176654278


##########
conf/cassandra.yaml:
##########
@@ -566,6 +566,48 @@ commitlog_segment_size: 32MiB
 #
 # flush_compression: fast
 
+#
+sstable_compressor:

Review Comment:
   @Claudenw this should be `sstable_compression`. We have `commitlog_compression` as well as `hints_compression`. We have even `internode_compression` and `flush_compression`. No reason to have `sstable_compressor` instead of `sstable_compression`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1176678110


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),

Review Comment:
   @Claudenw what if names of these classes are changed of we make mistake typing them out? Would not it be better if we use `LZ4Compressor.class.getName()` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182299814


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {

Review Comment:
   I believe that I have fixed the braces and space changes requested through out this review.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Claudenw commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "Claudenw (via GitHub)" <gi...@apache.org>.
Claudenw commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1182303580


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -97,128 +109,185 @@
     // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
-    public static CompressionParams fromMap(Map<String, String> opts)
-    {
-        Map<String, String> options = copyOptions(opts);
-
-        String sstableCompressionClass;
-
-        if (!opts.isEmpty() && isEnabled(opts) && !containsSstableCompressionClass(opts))
-            throw new ConfigurationException(format("Missing sub-option '%s' for the 'compression' option.", CLASS));
 
-        if (!removeEnabled(options))
-        {
-            sstableCompressionClass = null;
+    public enum CompressorType
+    {
+        lz4("LZ4Compressor"),
+        none(null),
+        noop("NoopCompressor"),
+        snappy("SnappyCompressor"),
+        deflate("DeflateCompressor"),
+        zstd("ZstdCompressor");
 
-            if (!options.isEmpty())
-                throw new ConfigurationException(format("If the '%s' option is set to false no other options must be specified", ENABLED));
-        }
-        else
-        {
-            sstableCompressionClass = removeSstableCompressionClass(options);
+        String className;
+        CompressorType(String className) {
+            this.className = className;
         }
 
-        int chunkLength = removeChunkLength(options);
-        double minCompressRatio = removeMinCompressRatio(options);
-
-        CompressionParams cp = new CompressionParams(sstableCompressionClass, options, chunkLength, minCompressRatio);
-        cp.validate();
-
-        return cp;
+        static CompressorType forClass(String name) {
+            for (CompressorType type : CompressorType.values()) {
+                if (Objects.equal(type.className, name)) {
+                    return type;
+                }
+            }
+            return null;
+        }
     }
 
-    public Class<? extends ICompressor> klass()
+    public static CompressionParams defaultParams()
     {
-        return sstableCompressor.getClass();
+        return fromParameterizedClass(DatabaseDescriptor.getSSTableCompressionOptions());
     }
 
-    public static CompressionParams noCompression()
+    public static CompressionParams fromParameterizedClass(ParameterizedClass options)
     {
-        return new CompressionParams((ICompressor) null, DEFAULT_CHUNK_LENGTH, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        if (options == null)
+        {
+            return CassandraRelevantProperties.DETERMINISM_SSTABLE_COMPRESSION_DEFAULT.getBoolean()
+                   ? DEFAULT
+                   : noCompression();
+        }
 
-    // The shorthand methods below are only used for tests. They are a little inconsistent in their choice of
-    // parameters -- this is done on purpose to test out various compression parameter combinations.
+        if (options.parameters != null && options.parameters.containsKey(SSTABLE_COMPRESSION))
+            throw new ConfigurationException(format("The '%s' option must not be used with the ParameterizedClass constructor",
+                                                    SSTABLE_COMPRESSION));
 
-    @VisibleForTesting
-    public static CompressionParams snappy()
-    {
-        return snappy(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(options.class_name, options.parameters == null ? Collections.emptyMap() : copyOptions(options.parameters));
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength)
+    private static String invalidValue( String param, Object value)
     {
-        return snappy(chunkLength, 1.1);
+        return format("Invalid '%s' value for the 'compression' option: %s", param, value);
     }
 
-    @VisibleForTesting
-    public static CompressionParams snappy(int chunkLength, double minCompressRatio)
-    {
-        return new CompressionParams(SnappyCompressor.instance, chunkLength, calcMaxCompressedLength(chunkLength, minCompressRatio), minCompressRatio, Collections.emptyMap());
+    private static String invalidValue( String param, String extraText, Object value) {
+        return format("Invalid '%s' value for the 'compression' option.  %s: %s", param, extraText, value );
     }
 
-    @VisibleForTesting
-    public static CompressionParams deflate()
+    public static CompressionParams fromMap(Map<String, String> opts)
     {
-        return deflate(DEFAULT_CHUNK_LENGTH);
-    }
+        Map<String, String> options = copyOptions(opts);
 
-    @VisibleForTesting
-    public static CompressionParams deflate(int chunkLength)
-    {
-        return new CompressionParams(DeflateCompressor.instance, chunkLength, Integer.MAX_VALUE, 0.0, Collections.emptyMap());
-    }
+        String sstableCompressionClass = removeSstableCompressionClass(options);
 
-    @VisibleForTesting
-    public static CompressionParams lz4()
-    {
-        return lz4(DEFAULT_CHUNK_LENGTH);
+        return fromClassAndOptions(sstableCompressionClass, options);
     }
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength)
+    private static CompressionParams fromClassAndOptions(String sstableCompressionClass, Map<String,String> options)
     {
-        return lz4(chunkLength, chunkLength);
-    }
+        boolean enabled = true;
 
-    @VisibleForTesting
-    public static CompressionParams lz4(int chunkLength, int maxCompressedLength)
-    {
-        return new CompressionParams(LZ4Compressor.create(Collections.emptyMap()), chunkLength, maxCompressedLength, calcMinCompressRatio(chunkLength, maxCompressedLength), Collections.emptyMap());
-    }
+        if (!removeEnabled(options))
+        {
+            enabled = false;
+        }
 
-    public static CompressionParams zstd()
-    {
-        return zstd(DEFAULT_CHUNK_LENGTH);
+        int chunk_length_in_kb = removeChunkLength(options);
+
+        // figure out how we calculate the max_compressed_length and min_compress_ratio
+        if (options.containsKey(MIN_COMPRESS_RATIO)  && options.containsKey(MAX_COMPRESSED_LENGTH))
+        {
+            throw new ConfigurationException("Can not specify both 'min_compress_ratio' and 'max_compressed_length' for the compressor parameters.");
+        }
+
+        // calculate the max_compressed_length and min_compress_ratio
+        int max_compressed_length;
+        double min_compress_ratio;
+        String max_compressed_length_str = options.remove( MAX_COMPRESSED_LENGTH);
+        if (!StringUtils.isBlank(max_compressed_length_str))
+        {
+            try
+            {
+                max_compressed_length = new DataStorageSpec.IntKibibytesBound(max_compressed_length_str).toKibibytes();
+                validateMaxCompressedLength( max_compressed_length, chunk_length_in_kb);
+                min_compress_ratio = CompressionParams.calcMinCompressRatio(chunk_length_in_kb, max_compressed_length);
+            } catch (IllegalArgumentException e) {
+                throw new ConfigurationException(invalidValue( MAX_COMPRESSED_LENGTH, e.getMessage(), max_compressed_length_str ));
+            }
+        }
+        else
+        {
+            min_compress_ratio = removeMinCompressRatio(options);
+            validateMinCompressRatio( min_compress_ratio );
+            max_compressed_length =  CompressionParams.calcMaxCompressedLength(chunk_length_in_kb,min_compress_ratio);
+        }
+
+        // try to set compressor type
+        CompressorType compressorType = StringUtils.isEmpty(sstableCompressionClass)?CompressorType.lz4:null;
+        if (compressorType == null)
+        {
+            try
+            {
+                compressorType = CompressorType.valueOf(sstableCompressionClass);
+            }
+            catch (IllegalArgumentException expected)
+            {
+                compressorType = CompressorType.forClass(sstableCompressionClass);
+            }
+        }
+
+        CompressionParams cp = null;
+        if (compressorType != null)
+        {
+
+            switch (compressorType)

Review Comment:
   Reworked to remove switch and clean up the process.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184988923


##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {

Review Comment:
   nit: brace on newline.



##########
test/unit/org/apache/cassandra/schema/CompressionParamsTest.java:
##########
@@ -0,0 +1,537 @@
+/*
+ * 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.cassandra.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.NoopCompressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionParamsTest
+{
+    //private ParameterizedClass options;
+    //private CompressionParams params;
+
+
+    private static  ParameterizedClass emptyParameterizedClass() {
+        return new ParameterizedClass(null, new HashMap<>());
+    }
+
+    @Test
+    public void additionalParamsTest() {
+        // no map

Review Comment:
   nit: brace on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] michaelsembwever commented on a diff in pull request #2254: Cassandra 12937 sstable compression rebased to trunk

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on code in PR #2254:
URL: https://github.com/apache/cassandra/pull/2254#discussion_r1184985305


##########
src/java/org/apache/cassandra/schema/CompressionParams.java:
##########
@@ -402,29 +411,47 @@ private static Integer parseChunkLength(String chLengthKB) throws ConfigurationE
      */
     private static int removeChunkLength(Map<String, String> options)
     {
-        if (options.containsKey(CHUNK_LENGTH_IN_KB))
-        {
-            if (options.containsKey(CHUNK_LENGTH_KB))
+        Integer chunk_length_in_kb = null;
+        String key = null;
+        if (options.containsKey(CHUNK_LENGTH)) {
+            key = CHUNK_LENGTH;
+            String value = options.remove(CHUNK_LENGTH);
+            try
             {
-                throw new ConfigurationException(format("The '%s' option must not be used if the chunk length is already specified by the '%s' option",
-                                                        CHUNK_LENGTH_KB,
-                                                        CHUNK_LENGTH_IN_KB));
+                chunk_length_in_kb =  new DataStorageSpec.IntKibibytesBound(value).toKibibytes();
+            } catch (IllegalArgumentException e) {

Review Comment:
   nit: braces on newline.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org