Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Duplicate
-
None
-
None
-
All
-
None
Description
When migrate data from 2.2.19 to 3.11.17, I encountered the following exception and the migration fails.
ERROR [main] 2024-04-25 19:41:22,996 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/mutation3085092904780349005dat. This may be caused by replaying a mutation against a table with the same name but incompatible schema. Exception follows: org.apache.cassandra.serializers.MarshalException: Expected 4 or 0 byte int (2) 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)
Reproduce
1. Start up single node cassandra-2.2.19 with default configuration and execute the following commands
CREATE KEYSPACE ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; CREATE TABLE ks.tb (c0 INT,c2 TEXT, PRIMARY KEY (c0)); INSERT INTO ks.tb (c0, c2) VALUES (1,'BB'); ALTER TABLE ks.tb DROP c2 ; ALTER TABLE ks.tb ADD c2 INT ;
2. Stop the 2.2 node
bin/nodetool -h ::FFFF:127.0.0.1 flush bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon;
3. Copy the data to 3.11.17 folder and start up, it will expose the following exception during the start up process. The node cannot start up.
ERROR [main] 2024-04-25 19:41:22,996 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/mutation3085092904780349005dat. This may be caused by replaying a mutation against a table with the same name but incompatible schema. Exception follows: org.apache.cassandra.serializers.MarshalException: Expected 4 or 0 byte int (2) 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)
I have attached the system.log and data.tar.gz. (use 3.11.17 to start up with this data can directly expose the error).
Drain before upgrade
If drain before upgrade, the upgrade will succeed but the read fails with the following exception
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'}
System log
ERROR [ReadStage-2] 2024-04-27 02:18:17,105 AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread Thread[ReadStage-2,10,main] java.lang.RuntimeException: org.apache.cassandra.serializers.MarshalException: Expected 4 or 0 byte int (2) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2777) 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) Caused by: org.apache.cassandra.serializers.MarshalException: Expected 4 or 0 byte int (2) at org.apache.cassandra.serializers.Int32Serializer.validate(Int32Serializer.java:42) at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:164) at org.apache.cassandra.db.marshal.AbstractType.validateIfFixedSize(AbstractType.java:427) at org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1488) 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) ... 5 common frames omitted
Attachments
Attachments
Issue Links
- duplicates
-
CASSANDRA-19590 Unexpected error deserializing mutation when upgrade from 2.2.19 to 3.0.30/3.11.17
- Patch Available