Klay created CASSANDRA-19591:
--------------------------------

             Summary: MarshalException when migrate data from 2.2.19 to 3.11.17
                 Key: CASSANDRA-19591
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19591
             Project: Cassandra
          Issue Type: Bug
          Components: Local/Commit Log
            Reporter: Klay
         Attachments: data.tar.gz, system.log

When migrate data from 2.2.19 to 3.11.17, I encountered the following exception 
and the migration fails.
{code:java}
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) 
{code}
h1. Reproduce

1. Start up single node cassandra-2.2.19 with default configuration and execute 
the following commands

 
{code:java}
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 ; {code}
2. Stop the 2.2 node
{code:java}
bin/nodetool -h ::FFFF:127.0.0.1 flush
bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon; {code}
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.
{code:java}
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) 
{code}
 

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



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