[
https://issues.apache.org/jira/browse/CASSANDRA-10377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14957386#comment-14957386
]
Vovodroid edited comment on CASSANDRA-10377 at 10/14/15 5:56 PM:
-----------------------------------------------------------------
Yes, in 2.1 function looks as
{code}
void recycleSegment(final CommitLogSegment segment)
{
boolean archiveSuccess =
CommitLog.instance.archiver.maybeWaitForArchiving(segment.getName());
activeSegments.remove(segment);
if (!archiveSuccess)
{
// if archiving (command) was not successful then leave the file
alone. don't delete or recycle.
discardSegment(segment, false);
return;
}
if (isCapExceeded() ||
!DatabaseDescriptor.getCommitLogSegmentRecyclingEnabled())
{
discardSegment(segment, true);
return;
}
logger.debug("Recycling {}", segment);
segmentManagementTasks.add(new Callable<CommitLogSegment>()
{
public CommitLogSegment call()
{
return segment.recycle();
}
});
}
{code}
So fix can be
{code}
if (!activeSegments.remove(segment))
{
logger.warn("segment {} not found in activeSegments queue", segment);
return;
}
{code}
was (Author: vovodroid):
Yes, function looks as
{code}
void recycleSegment(final CommitLogSegment segment)
{
boolean archiveSuccess =
CommitLog.instance.archiver.maybeWaitForArchiving(segment.getName());
activeSegments.remove(segment);
if (!archiveSuccess)
{
// if archiving (command) was not successful then leave the file
alone. don't delete or recycle.
discardSegment(segment, false);
return;
}
if (isCapExceeded() ||
!DatabaseDescriptor.getCommitLogSegmentRecyclingEnabled())
{
discardSegment(segment, true);
return;
}
logger.debug("Recycling {}", segment);
segmentManagementTasks.add(new Callable<CommitLogSegment>()
{
public CommitLogSegment call()
{
return segment.recycle();
}
});
}
{code}
So fix can be
{code}
if (!activeSegments.remove(segment))
{
logger.warn("segment {} not found in activeSegments queue", segment);
return;
}
{code}
> 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
> Assignee: Vovodroid
> Priority: Critical
> Attachments: 10377.patch
>
>
> 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)