[
https://issues.apache.org/jira/browse/CASSANDRA-21000?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18053876#comment-18053876
]
Stefan Miklosovic edited comment on CASSANDRA-21000 at 1/23/26 10:52 AM:
-------------------------------------------------------------------------
I literally can not comprehend what might be problematic about removing a
column from serialisation header upon compaction or upgradesstables (which is
just compaction camouflaged).
If I do this:
{code}
create table ks.tb (id int, val text);
insert into ks.tb (id) values (5);
alter table ks.tb drop val;
exit
nodetool flush
{code}
then "val" is NOT in a serialisation header. Based on what you wrote, then this
behavior is buggy too? There is no mentioning about column we have not been
writing into at all.
Similarly, when we compact, we are going to remove any trace of dropped column
/ data from SSTable. I just dont get what is wrong with removing dropped column
from serialisation header when that SSTable does not contain any data of such
column? - As that is exactly the case when we write an SSTable "populated" with
"inserts" which do not insert anything into a particular column either.
If it is not obvious from the patch, we are preserving the current state, that
is, dropped columns WILL be visible in serialisation header. We are not
changing anything there. The only time they are going to be removed is upon
compaction, reflecting the current state of the schema.
There is very nuanced difference in the behavior, we are doing this:
{code}
create table ks.tb (id int, val text);
insert into ks.tb (id) values (5);
alter table ks.tb drop val;
exit
nodetool flush
{code}
in this example, "val" will not be among the columns in the header. Why?
Because nobody ever wrote anything into "val" column.
Serialisation header is _lazily_ populated with columns, based on what columns
we "interacted" with while mutating.
{code}
create table ks.tb (id int, val text);
insert into ks.tb (id, val) values (5, 'abc'); // the difference
alter table ks.tb drop val;
exit
nodetool flush
{code}
Now, this WILL contain "val" in a serialisation header.
As said, we are not changing this behavior at all. It is all only about
compaction.
was (Author: smiklosovic):
I literally can not comprehend what might be problematic about removing a
column from serialisation header upon compaction or upgradesstables (which is
just compaction camouflaged).
If I do this:
{code}
create table ks.tb (id int, val text);
insert into ks.tb (id) values (5);
alter table ks.tb drop val;
exit
nodetool flush
{code}
then "val" is NOT in a serialisation header. Based on what you wrote, then this
behavior is buggy too? There is no mentioning about column we have not been
writing into at all.
Similarly, when we compact, we are going to remove any trace of dropped column
/ data from SSTable. I just dont get what is wrong with removing dropped column
from serialisation header when that SSTable does not contain any data of such
column? - As that is exactly the case when we write an SSTable "populated" with
"inserts" which do not insert anything into a particular column either.
If it is not obvious from the patch, we are preserving the current state, that
is, dropped columns WILL be visible in serialisation header. We are not
changing anything there. The only time they are going to be removed is upon
compaction, reflecting the current state of the schema.
> 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
> Components: Local/Compaction
> Reporter: Cameron Zemek
> Assignee: Stefan Miklosovic
> Priority: Normal
> Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x
>
> Time Spent: 1h
> 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]