[ 
https://issues.apache.org/jira/browse/CASSANDRA-19629?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Klay updated CASSANDRA-19629:
-----------------------------
    Description: 
When migrating data from 4.1.4 to 5.0 (commit: ccdeb12), the upgrade crashed 
with the following exception
{code:java}
ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
exception on startup, disk failure policy "stop"
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
    at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
    at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
    at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.AssertionError: null
    at 
org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
    at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
    at 
org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
    ... 10 common frames omitted
{code}
h1. Reproduce

Start up one 4.1.4 node using default configuration and execute the following 
command
{code:java}
CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE ks.tb (c1 INT, c2 INT, PRIMARY KEY (c1));
INSERT INTO ks.tb (c1, c2) VALUES (0,0);
ALTER TABLE ks.tb DROP c2 ;
ALTER TABLE ks.tb RENAME c1 TO c2;
{code}
Drain and upgrade to 5.0 (commit: ccdeb12)
{code:java}
bin/nodetool drain
bin/nodetool stopdaemon{code}
The upgrade would crash with the following exception
{code:java}
ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
exception on startup, disk failure policy "stop"
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
    at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
    at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
    at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.AssertionError: null
    at 
org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
    at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
    at 
org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
    ... 10 common frames omitted{code}
I attached a (1) data file. Use this file to start up 5.0 can easily reproduce 
it and (2)  system.log file.

  was:
When migrating data from 4.1.4 to 5.0 (commit: ccdeb12), the upgrade crashed 
with the following exception

 
{code:java}
ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
exception on startup, disk failure policy "stop"
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
    at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
    at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
    at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.AssertionError: null
    at 
org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
    at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
    at 
org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
    ... 10 common frames omitted
{code}
 
h1. Reproduce

Start up one 4.1.4 node using default configuration and execute the following 
command
{code:java}
CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE ks.tb (c1 INT, c2 INT, PRIMARY KEY (c1));
INSERT INTO ks.tb (c1, c2) VALUES (0,0);
ALTER TABLE ks.tb DROP c2 ;
ALTER TABLE ks.tb RENAME c1 TO c2;
{code}
Drain and upgrade to 5.0 (commit: ccdeb12)
{code:java}
bin/nodetool drain
bin/nodetool stopdaemon{code}
The upgrade would crash with the following exception
{code:java}
ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
exception on startup, disk failure policy "stop"
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
    at 
org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
    at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
    at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
    at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.AssertionError: null
    at 
org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
    at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
    at 
org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
    at 
org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
    at 
org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
    ... 10 common frames omitted{code}
I attached a (1) data file. Use this file to start up 5.0 can easily reproduce 
it and (2)  system.log file.


> Upgrade from 4.1.4 to 5.0 crashes with CorruptSSTableException
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-19629
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19629
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Klay
>            Priority: Normal
>         Attachments: data.tar.gz, system.log
>
>
> When migrating data from 4.1.4 to 5.0 (commit: ccdeb12), the upgrade crashed 
> with the following exception
> {code:java}
> ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
> DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> /home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
>     at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
>     at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
>     at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>     at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>     at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: java.lang.AssertionError: null
>     at 
> org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
>     at 
> org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
>     at 
> org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
>     at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
>     at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
>     ... 10 common frames omitted
> {code}
> h1. Reproduce
> Start up one 4.1.4 node using default configuration and execute the following 
> command
> {code:java}
> CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
> 'replication_factor' : 1 };
> CREATE TABLE ks.tb (c1 INT, c2 INT, PRIMARY KEY (c1));
> INSERT INTO ks.tb (c1, c2) VALUES (0,0);
> ALTER TABLE ks.tb DROP c2 ;
> ALTER TABLE ks.tb RENAME c1 TO c2;
> {code}
> Drain and upgrade to 5.0 (commit: ccdeb12)
> {code:java}
> bin/nodetool drain
> bin/nodetool stopdaemon{code}
> The upgrade would crash with the following exception
> {code:java}
> ERROR [SSTableBatchOpen:1] 2024-05-09 16:25:04,564 
> DefaultFSErrorHandler.java:129 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> /home/klay/system/cassandra/apache-cassandra-5.0/bin/../data/data/ks/tb-9f7e6da00e2011efa77a0bfeb6733ccc/nb-1-big
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:353)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReader.lambda$openAll$4(SSTableReader.java:414)
>     at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
>     at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
>     at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>     at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>     at java.base/java.lang.Thread.run(Thread.java:829)
> Caused by: java.lang.AssertionError: null
>     at 
> org.apache.cassandra.db.RegularAndStaticColumns$Builder.add(RegularAndStaticColumns.java:166)
>     at 
> org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:327)
>     at 
> org.apache.cassandra.io.sstable.format.StatsComponent.serializationHeader(StatsComponent.java:85)
>     at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:78)
>     at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
>     at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
>     ... 10 common frames omitted{code}
> I attached a (1) data file. Use this file to start up 5.0 can easily 
> reproduce it and (2)  system.log file.



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