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

Ke Han updated CASSANDRA-18108:
-------------------------------
    Description: 
When we upgrade Cassandra from 3.11.15 to 4.0.7, we found a data loss during 
the upgrade process. This bug can also be triggered if simply performing a 
system restart. 
h1. Steps to reproduce

Start a 3.11.15 or 4.0.7 Cassandra node using default configurations. Execute 
the following cqlsh commands.
{code:java}
CREATE KEYSPACE  ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) 
WITH speculative_retry = 'ALWAYS';
INSERT INTO ks.tb (c1, c2) VALUES (2,'val');
ALTER TABLE ks.tb DROP c2 ;
ALTER TABLE ks.tb RENAME c1 TO c2; {code}
Then execute a SELECT command, we get the correct data
{code:java}
cqlsh> SELECT *  FROM ks.tb;
 c2 | c3
----+------
  2 | null

(1 rows){code}
Flush and stop the Cassandra daemon.
{code:java}
bin/nodetool flush
bin/nodetool stopdaemon{code}
Then restart the node.
{code:java}
bin/cassandra{code}
Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost.
{code:java}
cqlsh> SELECT *  FROM ks.tb;
 c2 | c3
----+----

(0 rows){code}
 

During the node restart, we found an error log about initializing the table, 
but it didn't prevent the system from starting up.
{code:java}
INFO  [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - Initializing 
ks.tb
ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - 
Exception in thread Thread[SSTableBatchOpen:1,5,main]
java.lang.AssertionError: null
        at 
org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161)
        at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
        at java.lang.Thread.run(Thread.java:750) {code}
 

This bug can also be triggered if we perform an upgrade from 3.11.15 to 4.0.7 
and execute the SELECT command in the new version. (*The token_num 
configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, 
all the other configurations are using default values)

  was:
When we upgrade Cassandra from 3.11.14 to 4.0.7, we found a data loss during 
the upgrade process. This bug can also be triggered if simply performing a 
system restart. 
h1. Steps to reproduce

Start a 3.11.14 or 4.0.7 Cassandra node using default configurations. Execute 
the following cqlsh commands.
{code:java}
CREATE KEYSPACE  ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) 
WITH speculative_retry = 'ALWAYS';
INSERT INTO ks.tb (c1, c2) VALUES (2,'val');
ALTER TABLE ks.tb DROP c2 ;
ALTER TABLE ks.tb RENAME c1 TO c2; {code}
Then execute a SELECT command, we get the correct data
{code:java}
cqlsh> SELECT *  FROM ks.tb;
 c2 | c3
----+------
  2 | null

(1 rows){code}
Flush and stop the Cassandra daemon.
{code:java}
bin/nodetool flush
bin/nodetool stopdaemon{code}
Then restart the node.
{code:java}
bin/cassandra{code}
Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost.
{code:java}
cqlsh> SELECT *  FROM ks.tb;
 c2 | c3
----+----

(0 rows){code}
 

During the node restart, we found an error log about initializing the table, 
but it didn't prevent the system from starting up.
{code:java}
INFO  [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - Initializing 
ks.tb
ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - 
Exception in thread Thread[SSTableBatchOpen:1,5,main]
java.lang.AssertionError: null
        at 
org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161)
        at 
org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
        at java.lang.Thread.run(Thread.java:750) {code}
 

This bug can also be triggered if we perform an upgrade from 3.11.14 to 4.0.7 
and execute the SELECT command in the new version. (*The token_num 
configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, 
all the other configurations are using default values)


> Data loss after a system restart/upgrade (3.11.15)
> --------------------------------------------------
>
>                 Key: CASSANDRA-18108
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18108
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Ke Han
>            Priority: Normal
>
> When we upgrade Cassandra from 3.11.15 to 4.0.7, we found a data loss during 
> the upgrade process. This bug can also be triggered if simply performing a 
> system restart. 
> h1. Steps to reproduce
> Start a 3.11.15 or 4.0.7 Cassandra node using default configurations. Execute 
> the following cqlsh commands.
> {code:java}
> CREATE KEYSPACE  ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
> 'replication_factor' : 1 };
> CREATE TABLE IF NOT EXISTS ks.tb (c1 INT,c3 INT,c2 TEXT, PRIMARY KEY (c1 )) 
> WITH speculative_retry = 'ALWAYS';
> INSERT INTO ks.tb (c1, c2) VALUES (2,'val');
> ALTER TABLE ks.tb DROP c2 ;
> ALTER TABLE ks.tb RENAME c1 TO c2; {code}
> Then execute a SELECT command, we get the correct data
> {code:java}
> cqlsh> SELECT *  FROM ks.tb;
>  c2 | c3
> ----+------
>   2 | null
> (1 rows){code}
> Flush and stop the Cassandra daemon.
> {code:java}
> bin/nodetool flush
> bin/nodetool stopdaemon{code}
> Then restart the node.
> {code:java}
> bin/cassandra{code}
> Start cqlsh, and execute the same SELECT command. The data in ks.tb is lost.
> {code:java}
> cqlsh> SELECT *  FROM ks.tb;
>  c2 | c3
> ----+----
> (0 rows){code}
>  
> During the node restart, we found an error log about initializing the table, 
> but it didn't prevent the system from starting up.
> {code:java}
> INFO  [main] 2022-12-09 21:37:54,234 ColumnFamilyStore.java:432 - 
> Initializing ks.tb
> ERROR [SSTableBatchOpen:1] 2022-12-09 21:37:54,237 CassandraDaemon.java:244 - 
> Exception in thread Thread[SSTableBatchOpen:1,5,main]
> java.lang.AssertionError: null
>       at 
> org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:161)
>       at 
> org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:340)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:522)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:385)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReader$3.run(SSTableReader.java:570)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>       at java.lang.Thread.run(Thread.java:750) {code}
>  
> This bug can also be triggered if we perform an upgrade from 3.11.15 to 4.0.7 
> and execute the SELECT command in the new version. (*The token_num 
> configuration in 4.0.7 is modified to 16 for upgrade compatibility purposes, 
> all the other configurations are using default values)



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