[ https://issues.apache.org/jira/browse/CASSANDRA-12397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15528543#comment-15528543 ]
Stefania commented on CASSANDRA-12397: -------------------------------------- Reproduced in 3.0 HEAD by following the steps of CASSANDRA-11820: {code} cqlsh> CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; cqlsh> CREATE TABLE ks.test (a int PRIMARY KEY, b int); cqlsh> INSERT INTO ks.test (a, b) VALUES (1, 1); cqlsh> ALTER TABLE ks.test ALTER b TYPE BLOB; cqlsh> SELECT * from ks.test ; {code} At this point restarting the node will give the following exception: {code} INFO 06:00:03 Replaying /home/stefi/git/cstar/cassandra/bin/../data/commitlog/CommitLog-6-1475042184522.log, /home/stefi/git/cstar/cassandra/bin/../data/commitlog/CommitLog-6-1475042184523.log ERROR 06:00:03 Exiting due to error while processing commit log during initialization. org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: Unexpected error deserializing mutation; saved to /tmp/mutation4358621772356735283dat. 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 at org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:681) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:597) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:550) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:445) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:145) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:181) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:161) [main/:na] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:293) [main/:na] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:568) [main/:na] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:696) [main/:na] {code} Note that the mutation was actually serialized in an sstable that sstabledump can read without problems: {code} sstabledump data/data/ks/test-5b23e8e0854011e69e1cf96348c3ad08/mc-1-big-Data.db [ { "partition" : { "key" : [ "1" ], "position" : 0 }, "rows" : [ { "type" : "row", "position" : 18, "liveness_info" : { "tstamp" : "2016-09-28T05:57:05.612504Z" }, "cells" : [ { "name" : "b", "value" : "1" } ] } ] } ] {code} > 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)