cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Vovodroid (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-10377) AssertionError: attempted to delete non-existing file CommitLog
Date Sun, 20 Sep 2015 17:49:04 GMT

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

Vovodroid edited comment on CASSANDRA-10377 at 9/20/15 5:48 PM:
----------------------------------------------------------------

I added more logs, here is whole picture:
{code}
Commit log segment CommitLogSegment(CommitLog-5-1442725099391.log) is unused, force recycleSegment,
TID:86
Add task CommitLogSegmentManager$2@15a719ab
Commit log segment CommitLogSegment(CommitLog-5-1442725099391.log) is unused, recycleSegment,
TID:18
Add task CommitLogSegmentManager$2@4a81b2cf
segmentManagementTasks.take and run CommitLogSegmentManager$2@15a719ab
deleteWithConfirm [CommitLog-5-1442725099391.log] 
segmentManagementTasks.poll and run CommitLogSegmentManager$2@4a81b2cf
deleteWithConfirm [CommitLog-5-1442725099391.log] 
not exists CommitLog-5-1442725099391.log
{code}

So what happened?
1) Function *forceRecycleAll* in thread 86 decided that segment is unused and forced enqueuing
deletion task *15a719ab*
2) Function *discardCompletedSegments* in thread 18 decided that segment is unused and enqueued
deletion task *4a81b2cf*
3) Task *15a719ab* was dequeued and file was deleted.
4) Task *4a81b2cf* was dequeued and the same file was tried to be deleted.

Any thoughts? If it's normal behavior probably allow segment deletion to not fail if file
doesn't not exist ( call deleteWithConfirm(file, false, null) ?


was (Author: vovodroid):
I added more logs, here is whole picture:
{code}
Commit log segment CommitLogSegment(CommitLog-5-1442725099391.log) is unused, force recycleSegment,
TID:86
Add task CommitLogSegmentManager$2@15a719ab
Commit log segment CommitLogSegment(CommitLog-5-1442725099391.log) is unused, recycleSegment,
TID:18
Add task CommitLogSegmentManager$2@4a81b2cf
segmentManagementTasks.take and run CommitLogSegmentManager$2@15a719ab
deleteWithConfirm [CommitLog-5-1442725099391.log] 
segmentManagementTasks.poll and run CommitLogSegmentManager$2@4a81b2cf
deleteWithConfirm [CommitLog-5-1442725099391.log] 
not exists CommitLog-5-1442725099391.log
{code}

So what happened?
1) Function *forceRecycleAll* in thread 86 decided that segment is unused and forced enqueuing
deletion task *15a719ab*
2) Function *discardCompletedSegments* decided that segment is unused and enqueued deletion
task *4a81b2cf*
3) Task *15a719ab* was dequeued and file was deleted.
4) Task *4a81b2cf* was dequeued and the same file was tried to be deleted.

Any thoughts? If it's normal behavior probably allow segment deletion to not fail if file
doesn't not exist ( call deleteWithConfirm(file, false, null) ?

> AssertionError: attempted to delete non-existing file CommitLog
> ---------------------------------------------------------------
>
>                 Key: CASSANDRA-10377
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10377
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: CentOS 7.1/x64
>            Reporter: Vovodroid
>            Priority: Critical
>
> After several hours of script tests (create and drop users, keyspaces and tables) exception
is thrown:
> {code}
> ERROR 02:58:39 Failed managing commit log segments. Commit disk failure policy is stop;
terminating thread
> java.lang.AssertionError: attempted to delete non-existing file CommitLog-5-1442599226756.log
> 	at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:122) ~[main/:na]
> 	at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:149) ~[main/:na]
> 	at org.apache.cassandra.db.commitlog.CommitLogSegment.discard(CommitLogSegment.java:314)
~[main/:na]
> 	at org.apache.cassandra.db.commitlog.CommitLogSegmentManager$2.run(CommitLogSegmentManager.java:374)
~[main/:na]
> 	at org.apache.cassandra.db.commitlog.CommitLogSegmentManager$1.runMayThrow(CommitLogSegmentManager.java:155)
~[main/:na]
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) [main/:na]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> {code}
> I added some logs to *deleteWithConfirm* and it showed that this file really was deleted
by previous delete action, i.e. it was second attempt to delete the same log. Commit log with
next number exists in the same time, so log was switched.
> I disabled assert and it seems to have no no bad effect.



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

Mime
View raw message