cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Marcus Eriksson (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-10501) Failure to start up Cassandra when temporary compaction files are not all renamed after kill/crash (FSReadError)
Date Wed, 02 Dec 2015 09:30:11 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-10501?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Marcus Eriksson updated CASSANDRA-10501:
----------------------------------------
    Component/s: Local Write-Read Paths

> Failure to start up Cassandra when temporary compaction files are not all renamed after
kill/crash (FSReadError)
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-10501
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10501
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local Write-Read Paths
>         Environment: Cassandra 2.1.6
> Redhat Linux
>            Reporter: Mathieu Roy
>            Assignee: Marcus Eriksson
>              Labels: compaction, triage
>             Fix For: 2.1.12, 2.2.4
>
>
> We have seen an issue intermittently but repeatedly over the last few months where, after
exiting the Cassandra process, it fails to start with an FSReadError (stack trace below).
The FSReadError refers to a 'statistics' file for a  that doesn't exist, though a corresponding
temporary file does exist (eg. there is no /media/data/cassandraDB/data/clusteradmin/singleton_token-01a92ed069b511e59b2c53679a538c14/clusteradmin-singleton_token-ka-9-Statistics.db
file, but there is a /media/data/cassandraDB/data/clusteradmin/singleton_token-01a92ed069b511e59b2c53679a538c14/clusteradmin-singleton_token-tmp-ka-9-Statistics.db
file.)
> We tracked down the issue to the fact that the process exited with leftover compactions
and some of the 'tmp' files for the SSTable had been renamed to final files, but not all of
them - the issue happens if the 'Statistics' file is not renamed but others are. The scenario
we've seen on the last two occurrences involves the 'CompressionInfo' file being a final file
while all other files for the SSTable generation were left with 'tmp' names.
> When this occurs, Cassandra cannot start until the file issue is resolved; we've worked
around it by deleting the SSTable files from the same generation, both final and tmp, which
at least allows Cassandra to start. Renaming all files to either tmp or final names would
also work.
> We've done some debugging in Cassandra and have been unable to cause the issue without
renaming the files manually. The rename code at SSTableWriter.rename() looks like it could
result in this if the process exits in the middle of the rename, but in every occurrence we've
debugged through, the Set of components is ordered and Statistics is the first file renamed.
> However the comments in SSTableWriter.rename() suggest that the 'Data' file is meant
to be used as meaning the files were completely renamed. The method ColumnFamilyStore. removeUnfinishedCompactionLeftovers(),
however, will proceed assuming the compaction is complete if any of the component files has
a final name, and will skip temporary files when reading the list. If the 'Statistics' file
is temporary then it won't be read, and the defaults does not include a list of ancestors,
leading to the NullPointerException.
> It appears that ColumnFamilyStore. removeUnfinishedCompactionLeftovers() should perhaps
either ensure that all 'tmp' files are properly renamed before it uses them, or skip SSTable
files that don't have either the 'Data' or 'Statistics' file in final form.
> Stack trace: 
> {code}
> FSReadError in Failed to remove unfinished compaction leftovers (file: /media/data/cassandraDB/data/clusteradmin/singleton_token-01a92ed069b511e59b2c53679a538c14/clusteradmin-singleton_token-ka-9-Statistics.db).
 See log for details.
>         at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:617)
>         at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:302)
>         at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:536)
>         at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:625)
> Caused by: java.lang.NullPointerException
>         at org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:609)
>         ... 3 more
> Exception encountered during startup: java.lang.NullPointerException
> {code}



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

Mime
View raw message