Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 6707BCF6C for ; Fri, 7 Mar 2014 16:54:46 +0000 (UTC) Received: (qmail 11332 invoked by uid 500); 7 Mar 2014 16:54:45 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 11285 invoked by uid 500); 7 Mar 2014 16:54:44 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 11262 invoked by uid 99); 7 Mar 2014 16:54:43 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 Mar 2014 16:54:43 +0000 Date: Fri, 7 Mar 2014 16:54:43 +0000 (UTC) From: "Yuki Morishita (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-6818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13924048#comment-13924048 ] Yuki Morishita commented on CASSANDRA-6818: ------------------------------------------- I think Richard is right, and the same thing can happen to 2.0+. Considering fixing up to trunk, I think our option is No.2 on the above list. > SSTable references not released if stream session fails before it starts > ------------------------------------------------------------------------ > > Key: CASSANDRA-6818 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6818 > Project: Cassandra > Issue Type: Bug > Components: Core > Reporter: Richard Low > > I observed a large number of 'orphan' SSTables - SSTables that are in the data directory but not loaded by Cassandra - on a 1.1.12 node that had a large stream fail before it started. These orphan files are particularly dangerous because if the node is restarted and picks up these SSTables it could bring data back to life if tombstones have been GCed. To confirm the SSTables are orphan, I created a snapshot and it didn't contain these files. I can see in the logs that they have been compacted so should have been deleted. > The log entries for the stream are: > {{INFO [StreamStage:1] 2014-02-21 19:41:48,742 StreamOut.java (line 115) Beginning transfer to /10.0.0.1}} > {{INFO [StreamStage:1] 2014-02-21 19:41:48,743 StreamOut.java (line 96) Flushing memtables for [CFS(Keyspace='ks', ColumnFamily='cf1'), CFS(Keyspace='ks', ColumnFamily='cf2')]...}} > {{ERROR [GossipTasks:1] 2014-02-21 19:41:49,239 AbstractStreamSession.java (line 113) Stream failed because /10.0.0.1 died or was restarted/removed (streams may still be active in background, but further streams won't be started)}} > {{INFO [StreamStage:1] 2014-02-21 19:41:51,783 StreamOut.java (line 161) Stream context metadata [...] 2267 sstables.}} > {{INFO [StreamStage:1] 2014-02-21 19:41:51,789 StreamOutSession.java (line 182) Streaming to /10.0.0.1}} > {{INFO [Streaming to /10.0.0.1:1] 2014-02-21 19:42:02,218 FileStreamTask.java (line 99) Found no stream out session at end of file stream task - this is expected if the receiver went down}} > After digging in the code, here's what I think the issue is: > 1. StreamOutSession.transferRanges() creates a streaming session, which is registered with the failure detector in AbstractStreamSession's constructor. > 2. Memtables are flushed, potentially taking a long time. > 3. The remote node fails, convict() is called and the StreamOutSession is closed. However, at this time StreamOutSession.files is empty because it's still waiting for the memtables to flush. > 4. Memtables finish flusing, references are obtained to SSTables to be streamed and the PendingFiles are added to StreamOutSession.files. > 5. The first stream fails but the StreamOutSession isn't found so is never closed and the references are never released. > This code is more or less the same on 1.2 so I would expect it to reproduce there. I looked at 2.0 and can't even see where SSTable references are released when the stream fails. > Some possible fixes for 1.1/1.2: > 1. Don't register with the failure detector until after the PendingFiles are set up. I think this is the behaviour in 2.0 but I don't know if it was done like this to avoid this issue. > 2. Detect the above case in (e.g.) StreamOutSession.begin() by noticing the session has been closed with care to avoid double frees. > 3. Add some synchronization so closeInternal() doesn't race with setting up the session. -- This message was sent by Atlassian JIRA (v6.2#6252)