[ 
https://issues.apache.org/jira/browse/CASSANDRA-12397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15528843#comment-15528843
 ] 

Stefania commented on CASSANDRA-12397:
--------------------------------------

The problem is that {{PartitionUpdateSerializer}} does not serialize the column 
types because it calls 
{{UnfilteredRowIteratorSerializer.serializer.serialize(iter, null, out, 
version, update.rowCount())}}, which calls 
{{SerializationHeader.serializer.serializeForMessaging(header, selection, out, 
hasStatic)}}, which only serializes the column names. 

CASSANDRA-12461 fixes the exception that I've reproduced above because it 
recycles commit log segments in the shutdown hook, although there is still one 
CL file left.  However, this wouldn't cover the case of a crash just after a 
column is modified, in which case we would not be able to replay the mutation 
in the CL after startup. When a table is modified, a flush is scheduled in 
{{cfs.reload()}}, so there is a short window where the process may crash and be 
unable to restart. I'm not so sure what we can do about this other than 
changing CL format to include column types like we do for sstables.



> Altering a column's type breaks commitlog replay
> ------------------------------------------------
>
>                 Key: CASSANDRA-12397
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12397
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Carl Yeksigian
>            Assignee: Stefania
>
> When switching from a fixed-length column to a variable-length column, 
> replaying the commitlog on restart will have the same issue as 
> CASSANDRA-11820. Seems like it is related to the schema being flushed and 
> used when restarted, but commitlogs having been written in the old format.
> {noformat}
> org.apache.cassandra.db.commitlog.CommitLogReadHandler$CommitLogReadException:
>  Unexpected error deserializing mutation; saved to 
> /tmp/mutation4816372620457789996dat.  This may be caused by replaying a 
> mutation against a table with the same name but incompatible schema.  
> Exception follows: java.io.IOError: java.io.EOFException: EOF after 259 bytes 
> out of 3336
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:409)
>  [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:342)
>  [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:201)
>  [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReader.readAllFiles(CommitLogReader.java:84)
>  [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.replayFiles(CommitLogReplayer.java:139)
>  [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverFiles(CommitLog.java:177) 
> [main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.recoverSegmentsOnDisk(CommitLog.java:158)
>  [main/:na]
>         at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:316) 
> [main/:na]
>         at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:591)
>  [main/:na]
>         at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:720) 
> [main/:na]
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to