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

Yaron Gueta commented on CASSANDRA-8390:
----------------------------------------

Well, for me, it seems to be reproducible constantly. After investigating it 
for a while, it seems like the main issue is not the file lock but the result 
of it, which is, stopping Cassandra listeners (!) and making it unreachable.

I will begin with the file lock issue,
The file lock seems to be as a result of compaction that didn't release the 
handle yet, and it will be probably released after a while. This is not 
something which is new for Cassandra, after all, we can see a lot of messages 
like
{noformat}
ERROR [NonPeriodicTasks:1] 2015-01-02 11:53:13,291 SSTableDeletingTask.java:89 
- Unable to delete 
c:\apache-cassandra-2.1.2\data\data\system\schema_columns-296e9c049bec3085827dc17d3df2122a\system-schema_columns-ka-112-Data.db
 (it will be removed on server restart; we'll also retry after GC)
{noformat}
These file will be removed eventually, no harm done.

This specific behavior is part of SSTableDeletingTask.java

{code:title=SSTableDeletingTask.java|borderStyle=solid}
 public void run()
    {
        long size = referent.bytesOnDisk();

        if (tracker != null)
            tracker.notifyDeleting(referent);

        if (referent.readMeter != null)
            SystemKeyspace.clearSSTableReadMeter(referent.getKeyspaceName(), 
referent.getColumnFamilyName(), referent.descriptor.generation);

        // If we can't successfully delete the DATA component, set the task to 
be retried later: see above
        File datafile = new File(desc.filenameFor(Component.DATA));
        if (!datafile.delete())
        {
            logger.error("Unable to delete {} (it will be removed on server 
restart; we'll also retry after GC)", datafile);
            failedTasks.add(this);
            return;
        }
        // let the remainder be cleaned up by delete
        SSTable.delete(desc, Sets.difference(components, 
Collections.singleton(Component.DATA)));
        if (tracker != null)
            tracker.spaceReclaimed(size);
    }
{code}

The line:
{noformat}
 if (!datafile.delete())
{noformat}

does not throw an exception and the problem is being taken care of later on.

The problem is with the following section
{noformat}
SSTable.delete(desc, Sets.difference(components, 
Collections.singleton(Component.DATA)));
{noformat} 

This line leads to 
{code:title=FileUtils.java|borderStyle=solid}
public static void deleteWithConfirm(File file)
    {
        assert file.exists() : "attempted to delete non-existing file " + 
file.getName();
        if (logger.isDebugEnabled())
            logger.debug("Deleting {}", file.getName());
        try
        {
            Files.delete(file.toPath());
        }
        catch (IOException e)
        {
            throw new FSWriteError(e, file);
        }
    }
{code}

which throws an exception which no one catches and it goes to the 
UncaughtExceptionHandler.

The UncaughtExceptionHandler lead to
{code:title=FileUtils.java|borderStyle=solid}
public static void handleFSError(FSError e)
    {
        JVMStabilityInspector.inspectThrowable(e);
        switch (DatabaseDescriptor.getDiskFailurePolicy())
        {
            case stop_paranoid:
            case stop:
                StorageService.instance.stopTransports();
                break;
            case best_effort:
                // for both read and write errors mark the path as unwritable.
                BlacklistedDirectories.maybeMarkUnwritable(e.path);
                if (e instanceof FSReadError)
                {
                    File directory = 
BlacklistedDirectories.maybeMarkUnreadable(e.path);
                    if (directory != null)
                        Keyspace.removeUnreadableSSTables(directory);
                }
                break;
            case ignore:
                // already logged, so left nothing to do
                break;
            default:
                throw new IllegalStateException();
        }
    }
{code}
and the StorageService.instance.stopTransports(); is used. This, of course, 
makes Cassandra unreachable and the client to throw NoHostAvailableException.

The question is, file locks and deleting difficulties can happen, the decision 
to stop Cassandra listeners and make it unreachable is too risky to my opinion. 
Can it be handled that same way the other deleting tasks failures?

Thank you for the support.

> The process cannot access the file because it is being used by another process
> ------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-8390
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8390
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Ilya Komolkin
>            Assignee: Joshua McKenzie
>             Fix For: 2.1.3
>
>         Attachments: NoHostAvailableLogs.zip
>
>
> {code}21:46:27.810 [NonPeriodicTasks:1] ERROR o.a.c.service.CassandraDaemon - 
> Exception in thread Thread[NonPeriodicTasks:1,5,main]
> org.apache.cassandra.io.FSWriteError: java.nio.file.FileSystemException: 
> E:\Upsource_12391\data\cassandra\data\kernel\filechangehistory_t-a277b560764611e48c8e4915424c75fe\kernel-filechangehistory_t-ka-33-Index.db:
>  The process cannot access the file because it is being used by another 
> process.
>  
>                 at 
> org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:135) 
> ~[cassandra-all-2.1.1.jar:2.1.1]
>                 at 
> org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:121) 
> ~[cassandra-all-2.1.1.jar:2.1.1]
>                 at 
> org.apache.cassandra.io.sstable.SSTable.delete(SSTable.java:113) 
> ~[cassandra-all-2.1.1.jar:2.1.1]
>                 at 
> org.apache.cassandra.io.sstable.SSTableDeletingTask.run(SSTableDeletingTask.java:94)
>  ~[cassandra-all-2.1.1.jar:2.1.1]
>                 at 
> org.apache.cassandra.io.sstable.SSTableReader$6.run(SSTableReader.java:664) 
> ~[cassandra-all-2.1.1.jar:2.1.1]
>                 at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> ~[na:1.7.0_71]
>                 at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> ~[na:1.7.0_71]
>                 at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
>  ~[na:1.7.0_71]
>                 at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
>  ~[na:1.7.0_71]
>                 at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  ~[na:1.7.0_71]
>                 at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_71]
>                 at java.lang.Thread.run(Thread.java:745) [na:1.7.0_71]
> Caused by: java.nio.file.FileSystemException: 
> E:\Upsource_12391\data\cassandra\data\kernel\filechangehistory_t-a277b560764611e48c8e4915424c75fe\kernel-filechangehistory_t-ka-33-Index.db:
>  The process cannot access the file because it is being used by another 
> process.
>  
>                 at 
> sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86) 
> ~[na:1.7.0_71]
>                 at 
> sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) 
> ~[na:1.7.0_71]
>                 at 
> sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102) 
> ~[na:1.7.0_71]
>                 at 
> sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
>  ~[na:1.7.0_71]
>                 at 
> sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
>  ~[na:1.7.0_71]
>                 at java.nio.file.Files.delete(Files.java:1079) ~[na:1.7.0_71]
>                 at 
> org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:131) 
> ~[cassandra-all-2.1.1.jar:2.1.1]
>                 ... 11 common frames omitted{code}



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

Reply via email to