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

Klay reassigned CASSANDRA-19590:
--------------------------------

    Assignee: Klay

> Unexpected error deserializing mutation when upgrade from 2.2.19 to 
> 3.0.30/3.11.17
> ----------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-19590
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19590
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Klay
>            Assignee: Klay
>            Priority: Normal
>         Attachments: data.tar.gz, system.log
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> I am trying to upgrade from 2.2.19 to 3.0.30/3.11.17. I encountered the 
> following exception during the upgrade process and the 3.0.30/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. Reproduce1 (Flush before upgrade)
> Upgrade fails when replaying the commit log.
> 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 node (3.0.30 or 3.11.17)
> 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.
> h1. Reproduce2 (Drain before upgrade)
> If DRAIN before the upgrade, the upgrade can finish, but the read for the 
> table will fail and generate the following exception. It can be reproduced 
> upgrading from 2.2 to both 3.0.30 and 3.11.17
> h2. 2.2.19 => 3.0.30 with DRAIN
> Execute the same command in Reproduce1 in 2.2.19, when stop 2.2.19 node, use 
> drain
> {code:java}
> bin/nodetool -h ::FFFF:127.0.0.1 drain
> bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon{code}
> Start up the new version node (3.0.30), perform a read operation
> {code:java}
> ➜  apache-cassandra-3.0.30 bin/cqlsh
> Connected to Test Cluster at 127.0.0.1:9042.
> [cqlsh 5.0.1 | Cassandra 3.0.30 | CQL spec 3.4.0 | 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 [SharedPool-Worker-2] 2024-04-27 16:09:55,263 
> AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
> Thread[SharedPool-Worker-2,10,main]
> 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.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:554)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:510)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
>         at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.readStaticRow(AbstractSSTableIterator.java:148)
>         at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:104)
>         at 
> org.apache.cassandra.db.columniterator.SSTableIterator.<init>(SSTableIterator.java:51)
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:76)
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:366)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.metadata(LazilyInitializedUnfilteredRowIterator.java:53)
>         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:369)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$4.hasNext(UnfilteredPartitionIterators.java:220)
>         at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:92)
>         at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:292)
>         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:347)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1914)
>         at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2652)
>         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:109)
>         at java.lang.Thread.run(Thread.java:750) {code}
> h2. 2.2.19 => 3.11.17 with DRAIN
> Execute the same command in Reproduce1 in 2.2.19, when stop 2.2.19 node, use 
> drain
> {code:java}
> bin/nodetool -h ::FFFF:127.0.0.1 drain
> bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon{code}
> Start up the new version node (3.11.17), perform a read operation
> {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