cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Joshua McKenzie (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6736) Windows7 AccessDeniedException on commit log
Date Mon, 03 Mar 2014 18:30:21 GMT

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

Joshua McKenzie commented on CASSANDRA-6736:
--------------------------------------------

1st off - scratch what I said about the SSTableDeletingTask issue you saw above and 6283 -
we're not seeing that stack there.  We're seeing tasks enqueued as expected but not getting
file access errors.

2nd: Regarding the CommitLogSegment error: from the javadocs on MappedByteBuffer: "A mapped
byte buffer and the file mapping that it represents remain valid until the buffer itself is
garbage-collected."

It looks like we've seen this type of situation before - from SSTableDeletingTask.java comments:
"Deleting sstables is tricky because the mmapping might not have been finalized yet, and delete
will fail (on Windows) until it is (we only force the unmapping on SUN VMs). Additionally,
we need to make sure to delete the data file first, so on restart the others will be recognized
as GCable."

Our cleanup code in CommitLogSegment.java doesn't appear to take this into account:
{code:title=CommitLogSegment.java:148}
        close();  // FileUtils.clean(buffer) in this method
        if (deleteFile)
            FileUtils.deleteWithConfirm(logFile);
{code}

Unless FileUtils.clean(buffer) actually works on Windows - which SSTableDeletingTask would
imply does not - we may need to pursue deferring deletion attempts until after finalization
on buffer.  Creating a DeferredDeletionTask that'll fire after gc's and retry to deal with
this condition should be pretty straightforward.

Relevant bug from java bug db: http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4724038

> Windows7 AccessDeniedException on commit log 
> ---------------------------------------------
>
>                 Key: CASSANDRA-6736
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6736
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Windows 7, quad core, 8GB RAM, single Cassandra node, Cassandra
2.0.5 with leakdetect patch from CASSANDRA-6283
>            Reporter: Bill Mitchell
>            Assignee: Joshua McKenzie
>         Attachments: 2014-02-18-22-16.log
>
>
> Similar to the data file deletion of CASSANDRA-6283, under heavy load with logged batches,
I am seeing a problem where the Commit log cannot be deleted:
>  ERROR [COMMIT-LOG-ALLOCATOR] 2014-02-18 22:15:58,252 CassandraDaemon.java (line 192)
Exception in thread Thread[COMMIT-LOG-ALLOCATOR,5,main]
>  FSWriteError in C:\Program Files\DataStax Community\data\commitlog\CommitLog-3-1392761510706.log
> 	at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:120)
> 	at org.apache.cassandra.db.commitlog.CommitLogSegment.discard(CommitLogSegment.java:150)
> 	at org.apache.cassandra.db.commitlog.CommitLogAllocator$4.run(CommitLogAllocator.java:217)
> 	at org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:95)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> 	at java.lang.Thread.run(Unknown Source)
> Caused by: java.nio.file.AccessDeniedException: C:\Program Files\DataStax Community\data\commitlog\CommitLog-3-1392761510706.log
> 	at sun.nio.fs.WindowsException.translateToIOException(Unknown Source)
> 	at sun.nio.fs.WindowsException.rethrowAsIOException(Unknown Source)
> 	at sun.nio.fs.WindowsException.rethrowAsIOException(Unknown Source)
> 	at sun.nio.fs.WindowsFileSystemProvider.implDelete(Unknown Source)
> 	at sun.nio.fs.AbstractFileSystemProvider.delete(Unknown Source)
> 	at java.nio.file.Files.delete(Unknown Source)
> 	at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:116)
> 	... 5 more
> (Attached in 2014-02-18-22-16.log is a larger excerpt from the cassandra.log.)
> In this particular case, I was trying to do 100 million inserts into two tables in parallel,
one with a single wide row and one with narrow rows, and the error appeared after inserting
43,151,232 rows.  So it does take a while to trip over this timing issue.  
> It may be aggravated by the size of the batches. This test was writing 10,000 rows to
each table in a batch.  
> When I try switching the same test from using a logged batch to an unlogged batch, and
no such failure appears. So the issue could be related to the use of large, logged batches,
or it could be that unlogged batches just change the probability of failure.  



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message