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

Cameron Zemek commented on CASSANDRA-21000:
-------------------------------------------

> One I can think of is changing the type of a dropped column to something 
> incompatible with the underlying data.

[~brandon.williams] can you expand on this further. In the case of a dropped 
column the column does not appear in the Data.db . So what edge case is having 
a deleted column in the encoding stats needed for?

 

On side note there is protection to prevent changing to incompatible data type:
{code:java}
                if (!type.isSerializationCompatibleWith(droppedColumn.type))
                {
                    throw ire("Cannot re-add previously dropped column '%s' of 
type %s, incompatible with previous type %s",
                              name,
                              type.asCQL3Type(),
                              droppedColumn.type.asCQL3Type());
                } {code}
Not sure how to get into these edge cases where you have incompatible types for 
a column given these safeguards exist (schema mismatch and streaming perhaps?). 
What would that look like in the SSTable (assuming row would have to have a 
subset of columns to refer to the correct type for the conflicting column since 
the encoding stats would have both versions) in these edge cases?

> 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
>
> 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