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

Klay updated CASSANDRA-19590:
-----------------------------
    Description: 
I am trying to upgrade from 2.2.19 to 3.11.17. I encountered the following 
exception during the upgrade process and the 3.11.17 node cannot start up.
{code:java}
ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
due to error while processing commit log during initialization.
org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.  Exception 
follows: java.lang.AssertionError
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791) 
{code}
h1. Reproduce

This can be reproduced deterministically by 

1. Start up cassandra-2.2.19, singe node is enough (Using default configuration)

2. Execute the following commands in cqlsh
{code:java}
CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE ks.tb (c1 INT, c0 INT, PRIMARY KEY (c1));
INSERT INTO ks.tb (c1, c0) VALUES (0, 0);
ALTER TABLE ks.tb DROP c0 ;
ALTER TABLE ks.tb ADD c0 set<INT> ; 
{code}
3. Stop the old version.
{code:java}
bin/nodetool -h ::FFFF:127.0.0.1 flush
bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon{code}
4. Copy the data and start up the new version

Upgrade crashes with the following error
{code:java}
ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
due to error while processing commit log during initialization.
org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.  Exception 
follows: java.lang.AssertionError
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791){code}
I have attached the system.log when starting up the 3.11.17 node.

I also attached the data folder generated from the 2.2.19, start up 3.0.30 or 
3.11.17 with this data folder can directly expose the error.
h2. Upgrade from 2.2.19 to 3.0.30 will run into similar ERROR
{code:java}
ERROR 18:57:35 Exiting due to error while processing commit log during 
initialization.
org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8671047124793213031dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:746)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:671)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:624)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:508)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:190)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:187)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:168)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:342)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:636)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:837)
Caused by: java.lang.AssertionError: null
        at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:48)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1476)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1395)
        at 
org.apache.cassandra.db.LegacyLayout.getNextRow(LegacyLayout.java:845)
        at org.apache.cassandra.db.LegacyLayout.access$100(LegacyLayout.java:59)
        at 
org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:868)
        at 
org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:862)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.updateNextRow(RowAndDeletionMergeIterator.java:155)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNextInternal(RowAndDeletionMergeIterator.java:77)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:144)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:35)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.partitions.AbstractBTreePartition.build(AbstractBTreePartition.java:337)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate.fromIterator(PartitionUpdate.java:238)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate.fromPre30Iterator(PartitionUpdate.java:228)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserializePre30(PartitionUpdate.java:770)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:713)
        at 
org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:327)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:639)
        ... 8 common frames omitted {code}
h1. Drain before upgrade to remove commit log

If DRAIN before the upgrade, the upgrade can finish, but the read for the table 
will fail and generate the following exception
{code:java}
➜  apache-cassandra-3.11.17 bin/cqlsh
Connected to Test Cluster at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 3.11.17 | CQL spec 3.4.4 | Native protocol v4]
Use HELP for help.
cqlsh> SELECT * FROM ks.tb;
ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] 
message="Operation failed - received 0 responses and 1 failures" 
info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
'consistency': 'ONE'}{code}
System log
{code:java}
ERROR [ReadStage-2] 2024-04-27 02:10:24,085 
AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
Thread[ReadStage-2,10,main]
java.lang.AssertionError: null
        at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:44)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1489)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1407)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:555)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:511)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.readStaticRow(AbstractSSTableIterator.java:177)
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:113)
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.<init>(SSTableIterator.java:49)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:392)
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.metadata(LazilyInitializedUnfilteredRowIterator.java:58)
        at 
org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:60)
        at 
org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:34)
        at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
        at 
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374)
        at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186)
        at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$4.hasNext(UnfilteredPartitionIterators.java:233)
        at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:92)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:305)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76)
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:360)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2007)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2773)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:113)
        at java.lang.Thread.run(Thread.java:750) {code}

  was:
I am trying to upgrade from 2.2.19 to 3.11.17. I encountered the following 
exception during the upgrade process and the 3.11.17 node cannot start up.
{code:java}
ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
due to error while processing commit log during initialization.
org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.  Exception 
follows: java.lang.AssertionError
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791) 
{code}
h1. Reproduce

This can be reproduced deterministically by 

1. Start up cassandra-2.2.19, singe node is enough (Using default configuration)

2. Execute the following commands in cqlsh
{code:java}
CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE ks.tb (c1 INT,c2 TEXT, PRIMARY KEY (c1, c2));
ALTER TABLE ks.tb ADD c0 INT ;
INSERT INTO ks.tb (c0, c1, c2) VALUES (0,0,'RANDOM_STR');
CREATE INDEX idx ON ks.tb (c2);
ALTER TABLE ks.tb DROP c0 ;
ALTER TABLE ks.tb ADD c0 set<INT> ; {code}
3. Stop the old version.
{code:java}
bin/nodetool -h ::FFFF:127.0.0.1 flush
bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon{code}
4. Copy the data and start up the new version

Upgrade crashes with the following error
{code:java}
ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
due to error while processing commit log during initialization.
org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.  Exception 
follows: java.lang.AssertionError
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
        at 
org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791){code}
I have attached the system.log when starting up the 3.11.17 node.

I also attached the data folder generated from the 2.2.19, start up 3.0.30 or 
3.11.17 with this data folder can directly expose the error.
h2. Upgrade from 2.2.19 to 3.0.30 will run into similar ERROR
{code:java}
ERROR 18:57:35 Exiting due to error while processing commit log during 
initialization.
org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: 
Unexpected error deserializing mutation; saved to 
/tmp/mutation8671047124793213031dat.  This may be caused by replaying a 
mutation against a table with the same name but incompatible schema.
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:746)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:671)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:624)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:508)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:190)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:187)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:168)
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:342)
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:636)
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:837)
Caused by: java.lang.AssertionError: null
        at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:48)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1476)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1395)
        at 
org.apache.cassandra.db.LegacyLayout.getNextRow(LegacyLayout.java:845)
        at org.apache.cassandra.db.LegacyLayout.access$100(LegacyLayout.java:59)
        at 
org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:868)
        at 
org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:862)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.updateNextRow(RowAndDeletionMergeIterator.java:155)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNextInternal(RowAndDeletionMergeIterator.java:77)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:144)
        at 
org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:35)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.partitions.AbstractBTreePartition.build(AbstractBTreePartition.java:337)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate.fromIterator(PartitionUpdate.java:238)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate.fromPre30Iterator(PartitionUpdate.java:228)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserializePre30(PartitionUpdate.java:770)
        at 
org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:713)
        at 
org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:327)
        at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:639)
        ... 8 common frames omitted {code}
h1. Drain before upgrade to remove commit log

If DRAIN before the upgrade, the upgrade can finish, but the read for the table 
will fail and generate the following exception
{code:java}
➜  apache-cassandra-3.11.17 bin/cqlsh
Connected to Test Cluster at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 3.11.17 | CQL spec 3.4.4 | Native protocol v4]
Use HELP for help.
cqlsh> SELECT * FROM ks.tb;
ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] 
message="Operation failed - received 0 responses and 1 failures" 
info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
'consistency': 'ONE'}{code}
System log
{code:java}
ERROR [ReadStage-2] 2024-04-27 02:10:24,085 
AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
Thread[ReadStage-2,10,main]
java.lang.AssertionError: null
        at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:44)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1489)
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1407)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:555)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:511)
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.readStaticRow(AbstractSSTableIterator.java:177)
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:113)
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.<init>(SSTableIterator.java:49)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:392)
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.metadata(LazilyInitializedUnfilteredRowIterator.java:58)
        at 
org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:60)
        at 
org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:34)
        at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
        at 
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374)
        at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186)
        at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155)
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$4.hasNext(UnfilteredPartitionIterators.java:233)
        at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:92)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:305)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76)
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:360)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2007)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2773)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:113)
        at java.lang.Thread.run(Thread.java:750) {code}


> Unexpected error deserializing mutation when upgrade from 2.2.19 to 3.11.17
> ---------------------------------------------------------------------------
>
>                 Key: CASSANDRA-19590
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19590
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/Commit Log
>            Reporter: Klay
>            Priority: Normal
>         Attachments: data.tar.gz, system.log
>
>
> I am trying to upgrade from 2.2.19 to 3.11.17. I encountered the following 
> exception during the upgrade process and the 3.11.17 node cannot start up.
> {code:java}
> ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
> due to error while processing commit log during initialization.
> org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException:
>  Unexpected error deserializing mutation; saved to 
> /tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
> mutation against a table with the same name but incompatible schema.  
> Exception follows: java.lang.AssertionError
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
>         at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
>         at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
>         at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791) 
> {code}
> h1. Reproduce
> This can be reproduced deterministically by 
> 1. Start up cassandra-2.2.19, singe node is enough (Using default 
> configuration)
> 2. Execute the following commands in cqlsh
> {code:java}
> CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
> 'replication_factor' : 1 };
> CREATE TABLE ks.tb (c1 INT, c0 INT, PRIMARY KEY (c1));
> INSERT INTO ks.tb (c1, c0) VALUES (0, 0);
> ALTER TABLE ks.tb DROP c0 ;
> ALTER TABLE ks.tb ADD c0 set<INT> ; 
> {code}
> 3. Stop the old version.
> {code:java}
> bin/nodetool -h ::FFFF:127.0.0.1 flush
> bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon{code}
> 4. Copy the data and start up the new version
> Upgrade crashes with the following error
> {code:java}
> ERROR [main] 2024-04-25 18:46:10,496 JVMStabilityInspector.java:124 - Exiting 
> due to error while processing commit log during initialization.
> org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException:
>  Unexpected error deserializing mutation; saved to 
> /tmp/mutation8318204837345269856dat.  This may be caused by replaying a 
> mutation against a table with the same name but incompatible schema.  
> Exception follows: java.lang.AssertionError
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:471)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:404)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:251)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:132)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:137)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:189)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:170)
>         at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:331)
>         at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:630)
>         at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:791){code}
> I have attached the system.log when starting up the 3.11.17 node.
> I also attached the data folder generated from the 2.2.19, start up 3.0.30 or 
> 3.11.17 with this data folder can directly expose the error.
> h2. Upgrade from 2.2.19 to 3.0.30 will run into similar ERROR
> {code:java}
> ERROR 18:57:35 Exiting due to error while processing commit log during 
> initialization.
> org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: 
> Unexpected error deserializing mutation; saved to 
> /tmp/mutation8671047124793213031dat.  This may be caused by replaying a 
> mutation against a table with the same name but incompatible schema.
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:746)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:671)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:624)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:508)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:190)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:187)
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:168)
>         at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:342)
>         at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:636)
>         at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:837)
> Caused by: java.lang.AssertionError: null
>         at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:48)
>         at 
> org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1476)
>         at 
> org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1395)
>         at 
> org.apache.cassandra.db.LegacyLayout.getNextRow(LegacyLayout.java:845)
>         at 
> org.apache.cassandra.db.LegacyLayout.access$100(LegacyLayout.java:59)
>         at 
> org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:868)
>         at 
> org.apache.cassandra.db.LegacyLayout$2.computeNext(LegacyLayout.java:862)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.updateNextRow(RowAndDeletionMergeIterator.java:155)
>         at 
> org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNextInternal(RowAndDeletionMergeIterator.java:77)
>         at 
> org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:144)
>         at 
> org.apache.cassandra.db.rows.RowAndDeletionMergeIterator.computeNext(RowAndDeletionMergeIterator.java:35)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.partitions.AbstractBTreePartition.build(AbstractBTreePartition.java:337)
>         at 
> org.apache.cassandra.db.partitions.PartitionUpdate.fromIterator(PartitionUpdate.java:238)
>         at 
> org.apache.cassandra.db.partitions.PartitionUpdate.fromPre30Iterator(PartitionUpdate.java:228)
>         at 
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserializePre30(PartitionUpdate.java:770)
>         at 
> org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:713)
>         at 
> org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:327)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:639)
>         ... 8 common frames omitted {code}
> h1. Drain before upgrade to remove commit log
> If DRAIN before the upgrade, the upgrade can finish, but the read for the 
> table will fail and generate the following exception
> {code:java}
> ➜  apache-cassandra-3.11.17 bin/cqlsh
> Connected to Test Cluster at 127.0.0.1:9042.
> [cqlsh 5.0.1 | Cassandra 3.11.17 | CQL spec 3.4.4 | Native protocol v4]
> Use HELP for help.
> cqlsh> SELECT * FROM ks.tb;
> ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] 
> message="Operation failed - received 0 responses and 1 failures" 
> info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
> 'consistency': 'ONE'}{code}
> System log
> {code:java}
> ERROR [ReadStage-2] 2024-04-27 02:10:24,085 
> AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
> Thread[ReadStage-2,10,main]
> java.lang.AssertionError: null
>         at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:44)
>         at 
> org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1489)
>         at 
> org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1407)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:555)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:511)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
>         at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.readStaticRow(AbstractSSTableIterator.java:177)
>         at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:113)
>         at 
> org.apache.cassandra.db.columniterator.SSTableIterator.<init>(SSTableIterator.java:49)
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72)
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:392)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.metadata(LazilyInitializedUnfilteredRowIterator.java:58)
>         at 
> org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:60)
>         at 
> org.apache.cassandra.db.transform.RTBoundValidator.applyToPartition(RTBoundValidator.java:34)
>         at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
>         at 
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$4.hasNext(UnfilteredPartitionIterators.java:233)
>         at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:92)
>         at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:305)
>         at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
>         at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
>         at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
>         at 
> org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76)
>         at 
> org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:360)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2007)
>         at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2773)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>         at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:113)
>         at java.lang.Thread.run(Thread.java:750) {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org


Reply via email to