[
https://issues.apache.org/jira/browse/CASSANDRA-21000?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18038342#comment-18038342
]
Stefan Miklosovic edited comment on CASSANDRA-21000 at 11/14/25 1:58 PM:
-------------------------------------------------------------------------
Okay I took a deeper look into this and it is not entirely true that "deleted
columns are there for ever". When a column is dropped and I flush data into a
new SSTable, that dropped column is not there anymore. I was thinking that we
will add dropped into every SSTable but that is not true, I do not know why I
was thinking that ...
The example Cameron showed is correct, nodetool upgradesstables does not remove
dropped column _from SSTables which have it_.
I did a patch which removes all dropped columns from SSTable header but there
are actually two ways how to upgrade SSTables:
1) the first method is "nodetool upgradesstables" - this will require _live
Cassandra node_ against which this is performed.
2) the second method is doing it via "StandaloneUpgrader" which is used in
"./bin/sstableupgrade" - this does not need to have a node running, it just
reads it from a disk.
Method 1), when SSTables are upgraded against a live node, it basically creates
a new SSTable in a data dir for which it generates new ID. So if you rewrite it
and you have a snapshot as well (hence hard link), then it will create a new
SSTable in data dir but snapshot's SSTable will be intact.
Method 2), you can also point sstableupgrade to a specific snapshot of
keyspace and table and it will rewrite these (third optional parameter of
sstableupgrade is a snapshot name to rewrite SSTables in). That means that
after rewriting, the original SSTable in data dir will be intact.
I am not completely sure what approach is better. I do not think that we should
enable removing this in live data dir. This ticket also covers a use case when
they are _restoring a snapshot_. I can imagine that they would
1) take a snapshot on node A into directory 1
2) _now they need to rewrite SSTables to remove the dropped columns_ - this can
be done against snapshot directory itself (Method 2).
3) transfer SSTables to node B and import them without any other action needed
However 2) would inflate the snapshot size if some SSTables contain columns to
be dropped from header, because by rewriting, we would suddenly "materialize"
otherwise hardlinked file.
What method do you prefer?
for completeness:
{code}
CREATE KEYSPACE test2 WITH replication = {'class': 'SimpleStrategy',
'replication_factor': 1};
CREATE TABLE test2.drop_test(id int primary key, message text, col_to_delete
text);
INSERT INTO test2.drop_test(id, message, col_to_delete) VALUES (1, 'test',
'delete me');
ALTER TABLE test.drop_test DROP col_to_delete;
exit
./bin/nodetool flush
-> this will contain dropped columns in when inspected by sstablemetadata
on the other hand
CREATE KEYSPACE test2 WITH replication = {'class': 'SimpleStrategy',
'replication_factor': 1};
CREATE TABLE test2.drop_test(id int primary key, message text, col_to_delete
text);
INSERT INTO test2.drop_test(id, message, col_to_delete) VALUES (1, 'test',
'delete me');
exit
./bin/nodetool flush
ALTER TABLE test.drop_test DROP col_to_delete;
INSERT INTO test2.drop_test(id, message) VALUES (1, 'test');
exit
./bin/nodetool flush
-> this will not contain dropped column in the second SSTable
{code}
was (Author: smiklosovic):
Okay I took a deeper look into this and it is not entirely true that "deleted
columns are there for ever". When a column is dropped and I flush data into a
new SSTable, that dropped column is not there anymore. I was thinking that we
will add dropped into every SSTable but that is not true, I do not know why I
was thinking that ...
The example Cameron showed is correct, nodetool upgradesstables does not remove
dropped column _from SSTables which have it_.
I did a patch which removes all dropped columns from SSTable header but there
are actually two ways how to upgrade SSTables:
1) the first method is "nodetool upgradesstables" - this will require _live
Cassandra node_ against which this is performed.
2) the second method is doing it via "StandaloneUpgrader" which is used in
"./bin/sstableupgrade" - this does not need to have a node running, it just
reads it from a disk.
Method 1), when SSTables are upgraded against a live node, it basically creates
a new SSTable in a data dir for which it generates new ID. So if you rewrite it
and you have a snapshot as well (hence hard link), then it will create a new
SSTable in data dir but snapshot's SSTable will be intact.
Method 2), you can also point sstableupgrade to a specific snapshot of
keyspace and table and it will rewrite these (third optional parameter of
sstableupgrade is a snapshot name to rewrite SSTables in). That means that
after rewriting, the original SSTable in data dir will be intact.
I am not completely sure what approach is better. I do not think that we should
enable removing this in live data dir. This ticket also covers a use case when
they are _restoring a snapshot_. I can imagine that they would
1) take a snapshot on node A into directory 1
2) _now they need to rewrite SSTables to remove the dropped columns_ - this can
be done against snapshot directory itself (Method 2).
3) transfer SSTables to node B and import them without any other action needed
However 2) would inflate the snapshot size if some SSTables contain columns to
be dropped from header, because by rewriting, we would suddenly "materialize"
otherwise hardlinked file.
What method do you prefer?
> 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
> Assignee: Stefan Miklosovic
> 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]