[ https://issues.apache.org/jira/browse/CASSANDRA-6531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13857889#comment-13857889 ]
Pavel Yaskevich commented on CASSANDRA-6531: -------------------------------------------- [~jbellis] I think we should fsync twice in this case - first time whe the header is finalized and second time on close. > Failure to start after unclean shutdown - java.lang.IllegalArgumentException: > bufferSize must be positive > --------------------------------------------------------------------------------------------------------- > > Key: CASSANDRA-6531 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6531 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.0.3 with patches for CASSANDRA-6496 and > CASSANDRA-6284, built from source; Linux; XFS on the data disks; 5 data > disks; 6 nodes > Reporter: Nikolai Grigoriev > Assignee: Jonathan Ellis > Labels: compression > Fix For: 1.2.14, 2.0.5 > > Attachments: 6531.txt, cassandra_jstack.txt, system.log.gz, > system.log.gz > > > We had a severe power outage in the lab that resulted in unclean shutdown of > the Cassandra servers. After the power was back I tried to start the cluster. > Two out of 6 nodes cannot start because of this exception: > {code} > INFO 20:47:11,003 Initializing system.local > INFO [main] 2013-12-27 20:47:11,003 ColumnFamilyStore.java (line 251) > Initializing system.local > INFO 20:47:11,006 Opening > /hadoop/disk1/cassandra/data/system/local/system-local-jb-2478 (5836 bytes) > INFO [SSTableBatchOpen:1] 2013-12-27 20:47:11,006 SSTableReader.java (line > 223) Opening /hadoop/disk1/cassandra/data/system/local/system-local-jb-2478 > (5836 bytes) > INFO 20:47:11,006 Opening > /hadoop/disk4/cassandra/data/system/local/system-local-jb-2479 (144 bytes) > INFO [SSTableBatchOpen:2] 2013-12-27 20:47:11,006 SSTableReader.java (line > 223) Opening /hadoop/disk4/cassandra/data/system/local/system-local-jb-2479 > (144 bytes) > ERROR 20:47:12,366 Exception encountered during startup > java.lang.IllegalArgumentException: bufferSize must be positive > at > org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:67) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.<init>(CompressedRandomAccessReader.java:76) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:55) > at > org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1363) > at > org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:67) > at > org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1147) > at > org.apache.cassandra.db.RowIteratorFactory.getIterator(RowIteratorFactory.java:69) > at > org.apache.cassandra.db.ColumnFamilyStore.getSequentialIterator(ColumnFamilyStore.java:1526) > at > org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1645) > at > org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:137) > at > org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:236) > at > org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:1) > at > org.apache.cassandra.cql3.QueryProcessor.processInternal(QueryProcessor.java:255) > at > org.apache.cassandra.db.SystemKeyspace.getUnfinishedCompactions(SystemKeyspace.java:206) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:261) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:461) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:504) > {code} > Collecting the logs now, will attach to the issue in a moment. -- This message was sent by Atlassian JIRA (v6.1.5#6160)