[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-04-03 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


Great, +1.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0-v3.txt, 
> 6818-2.0-v4.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-04-02 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


[~yukim] looking better, a couple more issues though:

1. The timeoutExecutor shouldn't be static any more now it's per task
2. When the timeout occurs, inside StreamTransferTask.complete, 
timeoutTask.cancel is called while running the task. It's unclear from the 
java.util.concurrent code how safe this is. Did you test it? If it doesn't work 
a simple solution is to remove the task from timeoutTasks from within the 
runnable before calling complete.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0-v3.txt, 
> 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-04-02 Thread Yuki Morishita (JIRA)

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

Yuki Morishita commented on CASSANDRA-6818:
---

[~krummas] This can make SSTables that are compacted but not deleted. But for 
the SSTable in system/hints that cburroughs saw is different story since there 
should be no stream involved.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0-v3.txt, 
> 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-04-02 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-6818:


[~yukim] could this be the cause of CASSANDRA-6568 as well?

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0-v3.txt, 
> 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-24 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


[~yukim] I think this is a good approach, but I think there are some issues:

1. If the timeout runnable races with complete() then the SSTable could be 
freed twice and cause it to be incorrectly deleted. This is unlikely, but the 
impact is pretty bad.
2. The timeout should remove the file from files (and files should become a 
thread-safe data structure) and call session.taskCompleted if files.isEmpty().
3. I'm not sure, but I think timeoutExecutor should be shutdown otherwise there 
is a risk the non-daemon thread will stop the JVM exiting.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-24 Thread Yuki Morishita (JIRA)

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

Yuki Morishita commented on CASSANDRA-6818:
---

Committed 1.2. version to be released in 1.2.16.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-19 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


I reproduced the original problem in the same way, and the patch fixed it.

But I'm concerned about moving the reference release to 
StreamTransferTask.complete(). This is only called when the receiver sends a 
completed message, which may never occur. In this case I think the streaming 
session hangs, so we would never release the reference. However, the original 
behavior was also wrong because a retry could be requested after a successful 
transfer. The reference will have been released so the file may have been 
removed.

We could add a timeout to catch the case when the completed message gets lost. 
If the node requests a retry after this time, there may be no reference so the 
stream would fail. That's probably OK in this rare case though.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-14 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


Sure, I'll do a similar test next week.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-14 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6818:
---

Can you also look at the 2.0 patch, Richard?

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-14 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


I reproduced on 1.2 by inserting a sleep to delay the flushing. I killed a 
bootstrapping node during the sleep and references were leaked. With this 
patch, no references were leaked. So +1 for the 1.2 patch.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-11 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


I looked at the 1.2 patch, it looks fine. I'll see if I can reproduce the 
original issue to verify.

In StreamInSession.get, there is a minor memory leak - if another thread 
simultaneously creates the same session, the one that is discarded remains 
registered with the gossiper. This was present before, but we could easily fix 
it in this patch by delaying the registration until after the putIfAbsent 
succeeds.

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-11 Thread sankalp kohli (JIRA)

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

sankalp kohli commented on CASSANDRA-6818:
--

Sure. 

> 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
>Assignee: Yuki Morishita
> Fix For: 1.2.16, 2.0.7, 2.1 beta2
>
> Attachments: 6818-1.2.txt, 6818-2.0.txt
>
>
> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-07 Thread Yuki Morishita (JIRA)

[ 
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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-07 Thread Richard Low (JIRA)

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

Richard Low commented on CASSANDRA-6818:


We do see CASSANDRA-6503 as another source of orphans, but I think there are 
two separate issues. In particular, CASSANDRA-6503 creates orphans on failed 
stream in whereas this issue creates orphans on failed stream out. 

> 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)


[jira] [Commented] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts

2014-03-07 Thread Jason Brown (JIRA)

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

Jason Brown commented on CASSANDRA-6818:


We fixed this in CASSANDRA-6503, but that was only patched for 1.2 and above. 
As we're not really maintaining 1.1 any longer, I suggest you take the 1.2 
patch from that ticket and apply to 1.1 (it should be reasonably easily).

> 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)