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

Stefan Miklosovic edited comment on CASSANDRA-21000 at 1/21/26 8:32 PM:
------------------------------------------------------------------------

[~blambov] said earlier

"I don't see a reason why this should be the case; the purpose of the columns 
definition in the sstable header is to be able to read the data correctly. If 
data for a column is not present, its definition should not be needed."

This is indeed true. The way it works is a little bit complex. When an SSTable 
is going to be written, its serialisation header will contain only columns 
which were written into. Maybe it is not obvious but when I have an SSTable 
with 100 columns and I write a row which inserts data into 5 columns only then 
the serialisation header will contain only these 5 columns. 

There is ColumnsCollector class, as data are being flushed, it will call its 
update() method and it will set that column to be written to by flipping 
AtomicBoolean to true (by default it is false for each column). Then it will 
take only these columns into consideration for which its AtomicBoolean is true 
when constructing columns for serialisation header. For example check 
TrieMemtable.columns() method:

{code}
    RegularAndStaticColumns columns()
    {
        for (MemtableShard shard : shards)
            columnsCollector.update(shard.columnsCollector);
        return columnsCollector.get();
    }
{code}

Then check e.g. Flushing.createFlushWriter where this is passed to 
SerializationHeader constructor via {{flushSet.columns()}}.

This is actually quite clever way to do this, we are as efficient as possible 
here.

When it comes to deleted columns, upon flushing, we are writing these deletions 
too, of course, so that will also update "used columns" and hence they will be 
visible in the header. That also explains why we see deleted columns in header 
only in case we executed column dropping after we inserted some data into 
memtable. If there is a deleted column in a schema and we have a memtable full 
of inserts for undeleted columns only then such a (deleted) column will not be 
present in serialisation header.

However, when we go to compact a table (or two tables into one or similar), 
then we are not doing the same logic as upon flushing. We just take 
serialisation header as is, so even newly created SSTable does not contain any 
data of deleted columns, a deleted column is still present in its header. 

I think this really buggy behavior and it should be fixed. It has nothing to do 
with "different types" or anything like that.




was (Author: smiklosovic):
[~blambov] said earlier

"I don't see a reason why this should be the case; the purpose of the columns 
definition in the sstable header is to be able to read the data correctly. If 
data for a column is not present, its definition should not be needed."

This is indeed true. The way it works is a little bit complex. When an SSTable 
is going to be written, its serialisation header will contain only columns 
which were written into. Maybe it is not obvious but when I have an SSTable 
with 100 columns and I write a row which inserts data into 5 columns only then 
the serialisation header will contain only these 5 columns. 

There is ColumnsCollector class, as data are being flushed, it will call its 
update() method and it will set that column to be written to by flipping 
AtomicBoolean to true (by default it is false for each column). Then it will 
take only these columns into consideration for which its AtomicBoolean is true 
when constructing columns for serialisation header. For example check 
TrieMemtable.columns() method:

{code}
    RegularAndStaticColumns columns()
    {
        for (MemtableShard shard : shards)
            columnsCollector.update(shard.columnsCollector);
        return columnsCollector.get();
    }
{code}

Then check e.g. Flushing.createFlushWriter where this is passed to 
SerializationHeader constructor via {{flushSet.columns()}}.

This is actually quite clever way to do this, we are as efficient as possible 
here.

When it comes to deleted columns, upon flushing, we are writing these deletions 
too, of course, so that will also update "used columns" and hence they will be 
visible in the header. That also explains why we see deleted columns in header 
only in case we executed such deletions before flushing. If there is a deleted 
column in a schema and we have a memtable full of inserts only then such a 
(deleted) column will not be present in serialisation header.

However, when we go to compact a table (or two tables into one or similar), 
then we are not doing the same logic as upon flushing. We just take 
serialisation header as is, so even newly created SSTable does not contain any 
data of deleted columns, a deleted column is still present in its header. 

I think this really buggy behavior and it should be fixed. It has nothing to do 
with "different types" or anything like that.



> Deleted columns are forever part of SerializationHeader
> -------------------------------------------------------
>
>                 Key: CASSANDRA-21000
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-21000
>             Project: Apache Cassandra
>          Issue Type: Improvement
>            Reporter: Cameron Zemek
>            Priority: Normal
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> If you delete a column and rewrite the SSTable the column is removed from the 
> data, but the serialization header refers to the deleted column still. This 
> means if you drop a column and rewrite sstables (eg. nodetool upgradesstables 
> -a) and that column is not in use, you still can not import or load those 
> SSTables into another cluster without also having to add/drop columns.
>  
> {noformat}
> ~/.ccm/test/node1/data0/test $ ~/bin/cqlsh
> Connected to repairtest at 127.0.0.1:9042
> [cqlsh 6.2.0 | Cassandra 5.0.5-SNAPSHOT | CQL spec 3.4.7 | Native protocol v5]
> Use HELP for help.
> cqlsh> CREATE TABLE test.drop_test(id int primary key, message text, 
> col_to_delete text);
> cqlsh> INSERT INTO test.drop_test(id, message, col_to_delete) VALUES (1, 
> 'test', 'delete me');
> cqlsh> SELECT * FROM test.drop_test;
>  id | col_to_delete | message
> ----+---------------+---------
>   1 |     delete me |    test
> (1 rows)
> ~/.ccm/test/node1/data0/test $ ccm flush
> ~/.ccm/test/node1/data0/test $ cd drop_test-7a20f690ba8611f09c6c3125f1cbdf37
> ~/.ccm/test/node1/data0/test $ ls
> nb-1-big-CompressionInfo.db  nb-1-big-Digest.crc32  nb-1-big-Index.db       
> nb-1-big-Summary.db
> nb-1-big-Data.db             nb-1-big-Filter.db     nb-1-big-Statistics.db  
> nb-1-big-TOC.txt
> ~/.ccm/test/node1/data0/test $ /.ccm/repository/5.0.3/tools/bin/sstabledump 
> nb-1-big-Data.db
> [
>   {
>     "table kind" : "REGULAR",
>     "partition" : {
>       "key" : [ "1" ],
>       "position" : 0
>     },
>     "rows" : [
>       {
>         "type" : "row",
>         "position" : 18,
>         "liveness_info" : { "tstamp" : "2025-11-05T20:32:17.946616Z" },
>         "cells" : [
>           { "name" : "col_to_delete", "value" : "delete me" },
>           { "name" : "message", "value" : "test" }
>         ]
>       }
>     ]
>   }
> ]%
> ~/.ccm/test/node1/data0/test $ ~/bin/cqlsh
> Connected to repairtest at 127.0.0.1:9042
> [cqlsh 6.2.0 | Cassandra 5.0.5-SNAPSHOT | CQL spec 3.4.7 | Native protocol v5]
> Use HELP for help.
> cqlsh> ALTER TABLE test.drop_test DROP col_to_delete;
> cqlsh> SELECT * FROM test.drop_test;
>  id | message
> ----+---------
>   1 |    test
> (1 rows)
> ~/.ccm/test/node1/data0/test $ ccm node1 nodetool upgradesstables -- -a test 
> drop_test
> ~/.ccm/test/node1/data0/test $ ls
> nb-2-big-CompressionInfo.db  nb-2-big-Digest.crc32  nb-2-big-Index.db       
> nb-2-big-Summary.db
> nb-2-big-Data.db             nb-2-big-Filter.db     nb-2-big-Statistics.db  
> nb-2-big-TOC.txt
> ~/.ccm/test/node1/data0/test $ ~/.ccm/repository/5.0.3/tools/bin/sstabledump 
> nb-2-big-Data.db
> [
>   {
>     "table kind" : "REGULAR",
>     "partition" : {
>       "key" : [ "1" ],
>       "position" : 0
>     },
>     "rows" : [
>       {
>         "type" : "row",
>         "position" : 18,
>         "liveness_info" : { "tstamp" : "2025-11-05T20:32:17.946616Z" },
>         "cells" : [
>           { "name" : "message", "value" : "test" }
>         ]
>       }
>     ]
>   }
> ]%
> ~/.ccm/test/node1/data0/test $ 
> ~/.ccm/repository/5.0.3/tools/bin/sstablemetadata nb-2-big-Data.db | grep -E 
> 'StaticColumns|RegularColumns'
> StaticColumns:
> RegularColumns: col_to_delete:org.apache.cassandra.db.marshal.UTF8Type, 
> message:org.apache.cassandra.db.marshal.UTF8Type{noformat}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to