[ 
https://issues.apache.org/jira/browse/CASSANDRA-12937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17719227#comment-17719227
 ] 

Stefan Miklosovic edited comment on CASSANDRA-12937 at 5/4/23 9:23 AM:
-----------------------------------------------------------------------

I did another round of review:

When I do this
{code:java}
commitlog_compression:
  - class_name: lz4
    parameters:
      - enable: "true"
        lz4_compressor_type: "fast" {code}
This does not work: 
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Could not 
create Compression for type org.apache.cassandra.io.compress.lz4
    at 
org.apache.cassandra.schema.CompressionParams.parseCompressorClass(CompressionParams.java:338)
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:396)
    at 
org.apache.cassandra.db.commitlog.CommitLog$Configuration.<init>(CommitLog.java:630)
    at org.apache.cassandra.db.commitlog.CommitLog.<init>(CommitLog.java:107)
    at org.apache.cassandra.db.commitlog.CommitLog.construct(CommitLog.java:92)
    at org.apache.cassandra.db.commitlog.CommitLog.<clinit>(CommitLog.java:77) 
{code}
I would expect similar errors to be found when trying aliases for other 
_compression configuration properties.

Next, the default configuration for sstable_compression which is currently this 
does not work:
{code:java}
sstable_compression:
#   - class_name: lz4
#     parameters:
#       - enable: true
#         chunk_length: 16KiB
#         min_compress_ratio: 0.0
#         max_comrpessed_length: 16KiB
#         class_specific_parameter: value {code}
When I uncomment it like this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enable: true
         chunk_length: 16KiB
         min_compress_ratio: 0.0
         max_comrpessed_length: 16KiB
         class_specific_parameter: value {code}
First of all it says that:
{code:java}
Caused by: java.lang.ClassCastException: java.lang.Boolean cannot be cast to 
java.lang.CharSequence
    at 
org.apache.cassandra.schema.CompressionParams.copyOptions(CompressionParams.java:406)
    at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
    at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
    at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
    at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119) 
{code}
This means that it expects values to be strings, so I do it like this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enable: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value" {code}
But it says:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options enable
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
    at 
org.apache.cassandra.schema.CompressionParams.<init>(CompressionParams.java:260)
    at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:94)
    ... 15 common frames omitted {code}
I think this has to be changed to "enabled" as it is "enabled" in trunk too.

Next, it does not seem to me that the value of "lz4" is picked up because if I 
do this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value"
{code}
it does not say that the "class_specific_parameter" is invalid, check this: 
when I do
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value" {code}
it throws:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options class_specific_parameter
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
    at 
org.apache.cassandra.schema.CompressionParams.lambda$fromClassAndOptions$0(CompressionParams.java:228)
    at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:229)
    at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
    at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
    at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
    at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119)
    at 
org.apache.cassandra.cql3.statements.schema.TableAttributes.validate(TableAttributes.java:60)
    at 
org.apache.cassandra.cql3.statements.schema.CreateTableStatement.builder(CreateTableStatement.java:192)
    at 
org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse(CreateTableStatement.java:481)
    at org.apache.cassandra.schema.SchemaKeyspace.parse(SchemaKeyspace.java:276)
    at 
org.apache.cassandra.schema.SchemaKeyspace.<clinit>(SchemaKeyspace.java:90)
    ... 8 more 
{code}
So I remove that parameter so I start it like this:
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
{code}
This throws:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options max_comrpessed_length
        at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
        at 
org.apache.cassandra.schema.CompressionParams.lambda$fromClassAndOptions$0(CompressionParams.java:228)
        at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:229)
        at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
        at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
{code}
I think there is typo in max_comrpessed_length so it should be 
"max_compressed_length"

So I fix it and it says:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Can not 
specify both 'min_compress_ratio' and 'max_compressed_length' for the 
compressor parameters.
        at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:187)
        at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
        at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
        at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
        at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119)
{code}
So I remove "min_compress_ratio" and now it fails on:
{code:java}
Caused by: java.lang.RuntimeException: Cannot create CompressionParams for 
stored parameters
        at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:98)
        at 
org.apache.cassandra.io.sstable.format.CompressionInfoComponent.load(CompressionInfoComponent.java:53)
        at 
org.apache.cassandra.io.sstable.format.CompressionInfoComponent.maybeLoad(CompressionInfoComponent.java:38)
        at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:143)
        at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:60)
        at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
        ... 10 common frames omitted
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options class_specific_parameter
        at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
        at 
org.apache.cassandra.schema.CompressionParams.<init>(CompressionParams.java:260)
        at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:94)
{code}
I believe this is the consequence of saving parameter 
"class_specific_parameter" into an SSTable but the compressor will fail to 
parse it because it does not recognize it. That parameter, even unrecognized, 
was saved to SSTable but its reading upon start will fail because it does not 
know it. I think that the unknown parameters should not be stored into SSTable.

So I removed all data, started the node with this and it works:
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         max_compressed_length: "16KiB"
{code}
Also, do I understand it right that "max_compressed_length" is a parameter we 
are trying to introduce for the very first time? This is not configurable in 
trunk, it is just computed from this:
{code:java}
    static int calcMaxCompressedLength(int chunkLength, double minCompressRatio)
    {
        return (int) Math.ceil(Math.min(chunkLength / minCompressRatio, 
Integer.MAX_VALUE));
    }
{code}
So when I start this and write some data with the current patch in place and 
then I return to trunk and I start the node it throws:
{code:java}
org.apache.cassandra.exceptions.ConfigurationException: Invalid negative or 
null chunk_length_in_kb
        at 
org.apache.cassandra.schema.CompressionParams.validate(CompressionParams.java:528)
        at 
org.apache.cassandra.schema.CompressionParams.fromMap(CompressionParams.java:125)
        at 
org.apache.cassandra.schema.SchemaKeyspace.createTableParamsFromRow(SchemaKeyspace.java:1046)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchTable(SchemaKeyspace.java:1030)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchTables(SchemaKeyspace.java:990)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspace(SchemaKeyspace.java:950)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspacesWithout(SchemaKeyspace.java:940)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchNonSystemKeyspaces(SchemaKeyspace.java:928)
        at 
org.apache.cassandra.schema.DefaultSchemaUpdateHandler.reload(DefaultSchemaUpdateHandler.java:275)
        at 
org.apache.cassandra.schema.DefaultSchemaUpdateHandler.reset(DefaultSchemaUpdateHandler.java:287)
        at org.apache.cassandra.schema.Schema.loadFromDisk(Schema.java:157)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:283)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:764)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:888)
{code}
This means that what we wrote into a SSTable under new format - "chunk_length" 
- is not compatible anymore when old Cassandra reads it because it expects 
"chunk_length_in_kb" there. For that reason, I would still internally use 
"chunk_length_in_kb" when "chunk_length" is used. This has to be backward 
compatible. What if I want to use these SSTables in older version, like in 4.1?

I am also little bit puzzled by the git workflow here, all I see is that you 
"merged" trunk into that branch. When I checkout that I do not see the history 
so I am not sure how we want to squash and merge this. 

Do you think it is possible to prepare a branch which would contain 1 commit 
with all the work on top of the current trunk? Thanks. I am starting to lose 
the track what is going on.


was (Author: smiklosovic):
I did another round of review:

When I do this
{code:java}
commitlog_compression:
  - class_name: lz4
    parameters:
      - enable: "true"
        lz4_compressor_type: "fast" {code}
This does not work: 
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Could not 
create Compression for type org.apache.cassandra.io.compress.lz4
    at 
org.apache.cassandra.schema.CompressionParams.parseCompressorClass(CompressionParams.java:338)
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:396)
    at 
org.apache.cassandra.db.commitlog.CommitLog$Configuration.<init>(CommitLog.java:630)
    at org.apache.cassandra.db.commitlog.CommitLog.<init>(CommitLog.java:107)
    at org.apache.cassandra.db.commitlog.CommitLog.construct(CommitLog.java:92)
    at org.apache.cassandra.db.commitlog.CommitLog.<clinit>(CommitLog.java:77) 
{code}
I would expect similar errors to be found when trying aliases for other 
_compression configuration properties.

Next, the default configuration for sstable_compression which is currently this 
does not work:
{code:java}
sstable_compression:
#   - class_name: lz4
#     parameters:
#       - enable: true
#         chunk_length: 16KiB
#         min_compress_ratio: 0.0
#         max_comrpessed_length: 16KiB
#         class_specific_parameter: value {code}
When I uncomment it like this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enable: true
         chunk_length: 16KiB
         min_compress_ratio: 0.0
         max_comrpessed_length: 16KiB
         class_specific_parameter: value {code}
First of all it says that:
{code:java}
Caused by: java.lang.ClassCastException: java.lang.Boolean cannot be cast to 
java.lang.CharSequence
    at 
org.apache.cassandra.schema.CompressionParams.copyOptions(CompressionParams.java:406)
    at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
    at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
    at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
    at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119) 
{code}
This means that it expects values to be strings, so I do it like this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enable: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value" {code}
But it says:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options enable
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
    at 
org.apache.cassandra.schema.CompressionParams.<init>(CompressionParams.java:260)
    at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:94)
    ... 15 common frames omitted {code}
I think this has to be changed to "enabled" as it is "enabled" in trunk too.

Next, it does not seem to me that the value of "lz4" is picked up because if I 
do this:
{code:java}
sstable_compression:
   - class_name: lz4
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value"
{code}
it does not say that the "class_specific_parameter" is invalid, check this: 
when I do
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
         class_specific_parameter: "value" {code}
it throws:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options class_specific_parameter
    at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
    at 
org.apache.cassandra.schema.CompressionParams.lambda$fromClassAndOptions$0(CompressionParams.java:228)
    at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:229)
    at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
    at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
    at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
    at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119)
    at 
org.apache.cassandra.cql3.statements.schema.TableAttributes.validate(TableAttributes.java:60)
    at 
org.apache.cassandra.cql3.statements.schema.CreateTableStatement.builder(CreateTableStatement.java:192)
    at 
org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse(CreateTableStatement.java:481)
    at org.apache.cassandra.schema.SchemaKeyspace.parse(SchemaKeyspace.java:276)
    at 
org.apache.cassandra.schema.SchemaKeyspace.<clinit>(SchemaKeyspace.java:90)
    ... 8 more 
{code}
So I remove that parameter so I start it like this:
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         min_compress_ratio: "0.0"
         max_comrpessed_length: "16KiB"
{code}
This throws:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options max_comrpessed_length
        at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
        at 
org.apache.cassandra.schema.CompressionParams.lambda$fromClassAndOptions$0(CompressionParams.java:228)
        at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:229)
        at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
        at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
{code}
I think there is typo in max_comrpessed_length so it should be 
"max_compressed_length"

So I fix it and it says:
{code:java}
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Can not 
specify both 'min_compress_ratio' and 'max_compressed_length' for the 
compressor parameters.
        at 
org.apache.cassandra.schema.CompressionParams.fromClassAndOptions(CompressionParams.java:187)
        at 
org.apache.cassandra.schema.CompressionParams.fromParameterizedClass(CompressionParams.java:160)
        at 
org.apache.cassandra.schema.CompressionParams.defaultParams(CompressionParams.java:150)
        at 
org.apache.cassandra.schema.TableParams$Builder.<init>(TableParams.java:353)
        at org.apache.cassandra.schema.TableParams.builder(TableParams.java:119)
{code}
So I remove "min_compress_ratio" and now it fails on:
{code:java}
Caused by: java.lang.RuntimeException: Cannot create CompressionParams for 
stored parameters
        at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:98)
        at 
org.apache.cassandra.io.sstable.format.CompressionInfoComponent.load(CompressionInfoComponent.java:53)
        at 
org.apache.cassandra.io.sstable.format.CompressionInfoComponent.maybeLoad(CompressionInfoComponent.java:38)
        at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:143)
        at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:60)
        at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
        ... 10 common frames omitted
Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown 
compression options class_specific_parameter
        at 
org.apache.cassandra.schema.CompressionParams.createCompressor(CompressionParams.java:358)
        at 
org.apache.cassandra.schema.CompressionParams.<init>(CompressionParams.java:260)
        at 
org.apache.cassandra.io.compress.CompressionMetadata.open(CompressionMetadata.java:94)
{code}
I believe this is the consequence of saving parameter 
"class_specific_parameter" into an SSTable but the compressor will fail to 
parse it because it does not recognize it. That parameter, even unrecognized, 
was saved to SSTable but its reading upon start will fail because it does not 
know it. I think that the unknown parameters should not be stored into SSTable.

So I removed all data, started the node with this and it works:
{code:java}
sstable_compression:
   - class_name: LZ4Compressor
     parameters:
       - enabled: "true"
         chunk_length: "16KiB"
         max_compressed_length: "16KiB"
{code}
Also, do I understand it right that "max_compressed_length" is a parameter we 
are trying to introduce for the very first time? This is not configurable in 
trunk, it is just computed from this:
{code:java}
    static int calcMaxCompressedLength(int chunkLength, double minCompressRatio)
    {
        return (int) Math.ceil(Math.min(chunkLength / minCompressRatio, 
Integer.MAX_VALUE));
    }
{code}
So when I start this and write some data with the current patch in place and 
then I return to trunk and I start the node it throws:
{code:java}
org.apache.cassandra.exceptions.ConfigurationException: Invalid negative or 
null chunk_length_in_kb
        at 
org.apache.cassandra.schema.CompressionParams.validate(CompressionParams.java:528)
        at 
org.apache.cassandra.schema.CompressionParams.fromMap(CompressionParams.java:125)
        at 
org.apache.cassandra.schema.SchemaKeyspace.createTableParamsFromRow(SchemaKeyspace.java:1046)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchTable(SchemaKeyspace.java:1030)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchTables(SchemaKeyspace.java:990)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspace(SchemaKeyspace.java:950)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchKeyspacesWithout(SchemaKeyspace.java:940)
        at 
org.apache.cassandra.schema.SchemaKeyspace.fetchNonSystemKeyspaces(SchemaKeyspace.java:928)
        at 
org.apache.cassandra.schema.DefaultSchemaUpdateHandler.reload(DefaultSchemaUpdateHandler.java:275)
        at 
org.apache.cassandra.schema.DefaultSchemaUpdateHandler.reset(DefaultSchemaUpdateHandler.java:287)
        at org.apache.cassandra.schema.Schema.loadFromDisk(Schema.java:157)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:283)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:764)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:888)
{code}
This means that what we wrote into a SSTable under new format - "chunk_length" 
- is not compatible anymore when old Cassandra reads it because it expects 
"chunk_length_in_kb" there. For that reason, I would still internally use 
"chunk_length_in_kb" when "chunk_length" is used. This has to be backward 
compatible. What if I want to use these SSTables in older version, like in 4.1?

> Default setting (yaml) for SSTable compression
> ----------------------------------------------
>
>                 Key: CASSANDRA-12937
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12937
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Local/Config
>            Reporter: Michael Semb Wever
>            Assignee: Claude Warren
>            Priority: Low
>              Labels: AdventCalendar2021, lhf
>             Fix For: 5.x
>
>          Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> In many situations the choice of compression for sstables is more relevant to 
> the disks attached than to the schema and data.
> This issue is to add to cassandra.yaml a default value for sstable 
> compression that new tables will inherit (instead of the defaults found in 
> {{CompressionParams.DEFAULT}}.
> Examples where this can be relevant are filesystems that do on-the-fly 
> compression (btrfs, zfs) or specific disk configurations or even specific C* 
> versions (see CASSANDRA-10995 ).
> +Additional information for newcomers+
> Some new fields need to be added to {{cassandra.yaml}} to allow specifying 
> the field required for defining the default compression parameters. In 
> {{DatabaseDescriptor}} a new {{CompressionParams}} field should be added for 
> the default compression. This field should be initialized in 
> {{DatabaseDescriptor.applySimpleConfig()}}. At the different places where 
> {{CompressionParams.DEFAULT}} was used the code should call 
> {{DatabaseDescriptor#getDefaultCompressionParams}} that should return some 
> copy of configured {{CompressionParams}}.
> Some unit test using {{OverrideConfigurationLoader}} should be used to test 
> that the table schema use the new default when a new table is created (see 
> CreateTest for some example).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to