cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Bill Mitchell (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6736) Windows7 AccessDeniedException on commit log
Date Sat, 01 Mar 2014 20:57:20 GMT

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

Bill Mitchell commented on CASSANDRA-6736:
------------------------------------------

Off and on, I've been following the commentary in CASSANDRA-6283, but I opened this report
as my issue seems to be a different situation and path through the code.  In particular:
1.  I'm not seeing any reports from the leak detection patch, except in the log cited in CASSANDRA-6721.
2.  As I'm working in a test environment, I'm not dealing with any node repair issues.  With
only a single node I reboot it when it hangs, which clears the locks.  
3.  As these are only test cases, and I am frequently changing the schema, I'm am deleting
and recreating the keyspaces; so I would not notice any lingering data files.  And I gave
up after CASSANDRA-6721, and changed my test environment to use a new unique keyspace name,
except when I really want to exercise multiple runs against the same database.  
4.  As these are only tests, I disabled snapshots in the cassandra.yaml, so I'm not seeing
those locks.
5.  Similarly, after CASSANDRA-6721, I simply disabled saved key caching, so I'm not seeing
any issues around those files.  
All of which may mean I've avoided the issues mentioned in CASSANDRA-6283 and am hitting a
different set of issues.  

I can confirm that the issue here is not exclusive to the use of logged batches.  On Thursday
I ran into the same COMMIT_LOG_ALLOCATOR failure using smaller, overlapped unlogged batches
to the two tables (breaking the larger batch down into smaller segments, and overlapping the
one segment against one table, while the next segment is applied to the other table).  So
the use of large, logged batches just makes the problem likely enough for me to see it fairly
consistently. 

I have seen the file deletion failure in a couple of other situations where it seemed a secondary
result from an earlier failure.  If it helps, I will describe these below.   


> 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.1.5#6160)

Mime
View raw message