cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Tyler Hobbs (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-1752) repair leaving FDs unclosed
Date Tue, 30 Nov 2010 00:49:13 GMT

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

Tyler Hobbs commented on CASSANDRA-1752:
----------------------------------------

The temporary files that are streamed get deleted whenever the node receives a message saying
that the file was streamed successfully.  There isn't a need for SSTableReaders at all in
this case; only the names of the files produced by the anticompaction are needed for streaming.
 The fix here is to to simply close the SSTableWriter without opening an SSTableReader after
anticompaction and return a the list of filenames for use with streaming instead.  This way,
if waitForStreamCompletion() hangs indefinitely, there are no SSTRs around to keep the FDs
open.

> repair leaving FDs unclosed
> ---------------------------
>
>                 Key: CASSANDRA-1752
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1752
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Jonathan Ellis
>            Assignee: Tyler Hobbs
>             Fix For: 0.6.9
>
>         Attachments: 1752-0.6.txt
>
>
> "We noticed that after a `nodetool repair` was ran, several of our nodes reported high
disk usage; -- even one node hit 100% disk usage. After a restart of that node, disk usage
drop instantly by 80 gigabytes -- well that was confusing, but we quickly formed the theory
that Cassandra must of been holding open references to deleted file descriptors.
> "Later, i found this node as an example, it is using about 8-10 gigabytes more than it
should be -- 118 gigabytes reported by df, yet du reports only 106 gigabytes in the cassandra
directory (nothing else on the mahcine). As you can see from the lsof listing, it is holding
open FDs to files that no longer exist on the filesystem, and there are no open streams or
as far as I can tell other reasons for the deleted sstable to be open.
> "This seems to be related to running a repair, as we haven't seen it in any other situations
before."
> A quick check of FileStreamTask shows that the obvious base is covered:
> {code}
>         finally
>         {
>             try
>             {
>                 raf.close();
>             }
>             catch (IOException e)
>             {
>                 throw new AssertionError(e);
>             }
>         }
> {code}
> So it seems that either the transfer loop is never finishing to get to that finally block
(in which case why isn't it showing up in outbound streams?) or something else is the problem.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message