Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
None
-
Normal
Description
The system_schema.dropped_tables table is not flushed when schema is updated, this can result in SSTables not being loaded at startup and failure to start if the commit log contains mutations with the column.
Reproduce on cassandra-3.0 branch by starting a node and running following in cqlsh:
create keyspace dev WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; use dev; create table foo ( foo text primary key, bar text, baz text ); insert into foo (foo, bar, baz) values ('foo','this is bar', 'this is baz'); alter table foo drop baz;
Stop the node and restart, the following errors are raised and the node does not start:
ERROR 16:38:19 Exception in thread Thread[SSTableBatchOpen:1,5,main] java.lang.RuntimeException: Unknown column baz during deserialization at org.apache.cassandra.db.SerializationHeader$Component.toHeader(SerializationHeader.java:331) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:485) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:374) ~[main/:na] at org.apache.cassandra.io.sstable.format.SSTableReader$4.run(SSTableReader.java:533) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_60] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_60] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_60] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60] ... ERROR 16:38:19 Exiting due to error while processing commit log during initialization. org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: Unexpected error deserializing mutation; saved to /var/folders/r2/rkv1jz3j0j74r9s1zm5xx9wc0000gn/T/mutation5408885979635225676dat. This may be caused by replaying a mutation against a table with the same name but incompatible schema. Exception follows: java.lang.RuntimeException: Unknown column baz during deserialization at org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:633) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:556) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:509) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:404) [main/:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:151) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:189) [main/:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:169) [main/:na] at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:283) [main/:na] at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:551) [main/:na] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:679) [main/:na]
dropped_columns is not in the list of tables to flush, SchemaKeyspace.ALL.
It's a simple patch to add it, attached. The fix will need to go to 3.0, 3.1 and trunk AFAIK
however this will change the way the schema hash is calculated in SchemaKeyspace.calculateSchemaDigest() It looks like this would cause the nodes to announce a new version of the schema on (first) restart.
I currently donit understand all the implications of changing the schema hash, thoughts ?
Attachments
Attachments
Issue Links
- is duplicated by
-
CASSANDRA-11273 Unknown column failure during bootstrap
- Resolved