[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-19 Thread graham sanderson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13826912#comment-13826912
 ] 

graham sanderson edited comment on CASSANDRA-6275 at 11/19/13 8:39 PM:
---

Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also took 
a heap dump. I discovered what turned out to be 
https://issues.apache.org/jira/browse/CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in a heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked delete file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea, if this might be a problem (other than the 
memory leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue which 
was already completely removed and deallocated, in which case it would get GCed 
without close, presumably causing a file handle leak on the native side. Though 
I couldn't come up with any significantly convincing interactions that would 
cause this to happen without some very very unlucky things happening (and my 
knowledge of the google cache implementation was even more limited!), so this 
is unlikely the cause of this issue (especially if the issue doesn't happen in 
the 1.2.7+ branch), and unlikely also because there is no particular 
correlation with TTL I don't think


was (Author: graham sanderson):
Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also 
found a heap dump. I discovered what turned out to be 
https://issues.apache.org/jira/browse/CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in a heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked delete file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea, if this might be a problem (other than the 
memory leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue which 
was already completely removed and deallocated, in which case it would get GCed 
without close, presumably causing a file handle leak on the native side. Though 
I couldn't come up with any significantly convincing interactions that would 
cause this to happen without some very very unlucky things happening (and my 
knowledge of the google cache implementation was even more limited!), so this 
is unlikely the cause of this issue (especially if the issue doesn't happen in 
the 1.2.7+ branch).

 2.0.x leaks file handles
 

 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-19 Thread graham sanderson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13826912#comment-13826912
 ] 

graham sanderson edited comment on CASSANDRA-6275 at 11/19/13 8:39 PM:
---

Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also took 
a heap dump. I discovered what turned out to be CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in a heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked delete file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea, if this might be a problem (other than the 
memory leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue which 
was already completely removed and deallocated, in which case it would get GCed 
without close, presumably causing a file handle leak on the native side. Though 
I couldn't come up with any significantly convincing interactions that would 
cause this to happen without some very very unlucky things happening (and my 
knowledge of the google cache implementation was even more limited!), so this 
is unlikely the cause of this issue (especially if the issue doesn't happen in 
the 1.2.7+ branch), and unlikely also because there is no particular 
correlation with TTL I don't think


was (Author: graham sanderson):
Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also took 
a heap dump. I discovered what turned out to be 
https://issues.apache.org/jira/browse/CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in a heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked delete file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea, if this might be a problem (other than the 
memory leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue which 
was already completely removed and deallocated, in which case it would get GCed 
without close, presumably causing a file handle leak on the native side. Though 
I couldn't come up with any significantly convincing interactions that would 
cause this to happen without some very very unlucky things happening (and my 
knowledge of the google cache implementation was even more limited!), so this 
is unlikely the cause of this issue (especially if the issue doesn't happen in 
the 1.2.7+ branch), and unlikely also because there is no particular 
correlation with TTL I don't think

 2.0.x leaks 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-19 Thread graham sanderson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13826912#comment-13826912
 ] 

graham sanderson edited comment on CASSANDRA-6275 at 11/19/13 8:42 PM:
---

Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also took 
a heap dump. I discovered what turned out to be CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in the heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked deleted file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea if this might be a problem (other than the memory 
leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue that was 
already removed and drained, in which case it would get GCed without close, 
presumably causing a file handle leak on the native side. Though I couldn't 
come up with any significantly convincing interactions that would cause this to 
happen without some very very unlucky things happening (and my knowledge of the 
google cache implementation was even more limited!), so this is unlikely the 
cause of this issue (especially if the issue doesn't happen in the 1.2.7+ 
branch), because I think nearly all deleted data files are being leakd, and 
finally because there is no particular correlation with TTL.


was (Author: graham sanderson):
Yes I believe we can mitigate the problem in the OpCenter case, however it is a 
good test bed since it makes the problem easy to spot - note it seems to be 
worse under high read/write activity on tracked keyspaces/CFs, however that 
makes sense.

Note I was poking (somewhat blindly) thru the (2.0.2) code (partly out of 
interest) looking for what might be leaking these file handles, and I also took 
a heap dump. I discovered what turned out to be CASSANDRA-6358 which leaks 
FileDescriptors though their refCounts all seemed to be 0. In any case there 
weren't enough (total FileDescriptors - in a heap dump) to account for the 
problem. They were also for mem-mapped files (the ifile in SSTableReader) and 
none of the leaked delete file handles were mem-mapped (since they were 
compressed data files)

That said CASSANDRA-6358 was pinning the SSTableReaders in memory (since the 
Runnable was an anonymous inner class), so someone with more knowledge of the 
code might have a better idea, if this might be a problem (other than the 
memory leak)

I don't have an environment yet where I can easily build and install code 
changes, though we could downgrade our system test environment to 2.0.0 to see 
if we can reproduce the problem there - unsure if we can downgrade to 1.2.X 
easily given our current testing.

Note while I was looking at the code I came across CASSANDRA-... What 
caught my eye was the interaction between FileCacheService and RAR.deallocate, 
but more specifically related to the fact that this change, added a concurrent 
structure inside another separate concurrent structure, and it seemed like 
there might be a case where a RAR was recycled into a concurrent queue which 
was already completely removed and deallocated, in which case it would get GCed 
without close, presumably causing a file handle leak on the native side. Though 
I couldn't come up with any significantly convincing interactions that would 
cause this to happen without some very very unlucky things happening (and my 
knowledge of the google cache implementation was even more limited!), so this 
is unlikely the cause of this issue (especially if the issue doesn't happen in 
the 1.2.7+ branch), and unlikely also because there is no particular 
correlation with TTL I don't think

 2.0.x leaks file 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-19 Thread J. Ryan Earl (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13827162#comment-13827162
 ] 

J. Ryan Earl edited comment on CASSANDRA-6275 at 11/20/13 12:16 AM:


[~mishail] We (Graham Sanderson and I work together) added 
'file_cache_size_in_mb: 0' to cassandra.yaml on one of the nodes, and restarted 
that node plus another with the default (unspecified) file_cache_size_in_mb 
setting to run an A/B test.  Both nodes still leak file handles, however, the 
node with the default setting leaks much faster (about 3-4x the leak rate).

CASSANDRA-6283 appears to be an exact duplicate of this problem, Windows and 
Linux JVMs appear to exhibit the exact same file handle leak behavior.


was (Author: jre):
[~mishail] We (Graham Sanderson and I work together) added 
'file_cache_size_in_mb: 0' to cassandra.yaml on one of the nodes, and restart 
that node plus another with the default (unspecified) file_cache_size_in_mb 
setting to run an A/B test.  Both nodes still leak file handles, however, the 
node with the default setting leaks much faster (about 3-4x the leak rate).

CASSANDRA-6283 appears to be an exact duplicate of this problem, Windows and 
Linux JVMs appear to exhibit the exact same file handle leak behavior.

 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
Assignee: Michael Shuler
 Attachments: c_file-descriptors_strace.tbz, cassandra_jstack.txt, 
 leak.log, position_hints.tgz, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes
 Max data size unlimitedunlimitedbytes
 Max stack size10485760 unlimitedbytes
 Max core file size00bytes
 Max resident set  unlimitedunlimitedbytes
 Max processes 1024 unlimitedprocesses
 Max open files4096 4096 files
 Max locked memory unlimitedunlimitedbytes
 Max address space unlimitedunlimitedbytes
 Max file locksunlimitedunlimitedlocks
 Max pending signals   1463314633signals  
 Max msgqueue size 819200   819200   bytes
 Max nice priority 00   
 Max realtime priority 00   
 Max realtime timeout  unlimitedunlimitedus 
 {noformat}
 Looks like the problem is not in limits.
 Before load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 166
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 164
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 180
 {noformat}
 After load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 967
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 1766
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 2578
 {noformat}
 Most opened files have names like:
 {noformat}
 java  16890 cassandra 1636r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1637r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1638r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1639r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1640r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1641r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1642r  REG   

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-19 Thread graham sanderson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13827173#comment-13827173
 ] 

graham sanderson edited comment on CASSANDRA-6275 at 11/20/13 4:44 AM:
---

Note that this would tend to imply that I was wrong (at least about the 
particular code path), and the change in leak rate may be attributable to less 
throughput without the file cache. Note the leak rate does seem quite related 
to how hard we are hitting the server as mentioned before, so a threading bug 
elsewhere might be the cause.

Note nominally buffer in RAR should be volatile, but then any code path thru 
close where buffer's latest value is stale would end up calling deallocate 
anyway (at least in the case that file_cache_size_in_mb is off; I didn't think 
through the other case.

So given the finalizer fix - which we can try and build here to test out 
(unless someone has it pre-built) - seems to imply that it is just someone 
failing to call close() under load conditions.


was (Author: graham sanderson):
Note that this would tend to imply that I was wrong (at least about the 
particular code path), and the change in leak rate may be attributable to less 
throughput without the file cache. Note the leak rate does seem quite related 
to how hard we are hitting the server as mentioned before, so a threading bug 
elsewhere might be the cause.

Note nominally buffer in RAR should be volatile, but then any code path thru 
close where buffer's latest value is stale would end up calling deallocate 
anyway (at least in the case that file_cache_size_in_mb is off; I didn't think 
though the other case.

So given the finalizer fix - which we can try and build here to test out 
(unless someone has it pre-built) - seems to imply that it is just someone 
failing to call close() under load conditions.

 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
Assignee: Michael Shuler
 Attachments: c_file-descriptors_strace.tbz, cassandra_jstack.txt, 
 leak.log, position_hints.tgz, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes
 Max data size unlimitedunlimitedbytes
 Max stack size10485760 unlimitedbytes
 Max core file size00bytes
 Max resident set  unlimitedunlimitedbytes
 Max processes 1024 unlimitedprocesses
 Max open files4096 4096 files
 Max locked memory unlimitedunlimitedbytes
 Max address space unlimitedunlimitedbytes
 Max file locksunlimitedunlimitedlocks
 Max pending signals   1463314633signals  
 Max msgqueue size 819200   819200   bytes
 Max nice priority 00   
 Max realtime priority 00   
 Max realtime timeout  unlimitedunlimitedus 
 {noformat}
 Looks like the problem is not in limits.
 Before load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 166
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 164
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 180
 {noformat}
 After load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 967
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 1766
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 2578
 {noformat}
 Most opened files have names like:
 {noformat}
 java  16890 cassandra 1636r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1637r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1638r  REG 202,17  88724987 
 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-18 Thread J. Ryan Earl (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13825856#comment-13825856
 ] 

J. Ryan Earl edited comment on CASSANDRA-6275 at 11/18/13 10:26 PM:


We recently ran into this issue after upgrading to OpsCenter-4.0.0, it is quite 
easy to reproduce:
# Install Cassandra-2.0.2
# Install OpsCenter-4.0.0 on above cluster.

I upgraded OpsCenter on Friday, and by Sunday I had reached 1 Million open file 
handles.  I had to kill -9 the Cassandra processes as it wouldn't respond to 
sockets, DSC20 restart scripts reported successfully killing the processes but 
in fact did not.

{noformat}
[root@cassandra2 ~]# lsof -u cassandra|wc -l
175416
[root@cassandra2 ~]# lsof -u cassandra|grep -c OpsCenter
174474
{noformat}

Most of the handles show as deleted
{noformat}
[root@cassandra2 ~]# lsof -u cassandra|grep -c deleted
174449
{noformat}


was (Author: jre):
We recently ran into this issue after upgrading to OpsCenter-4.0.0, it is quite 
easy to reproduce:
# Install Cassandra-2.0.2
# Install OpsCenter-4.0.0 on above cluster.

I upgraded OpsCenter on Friday, and by Sunday I had reached 1 Million open file 
handles.  I had to kill -9 the Cassandra processes as it wouldn't respond to 
sockets, DSC20 restart scripts reported successfully killing the processes but 
in fact did not.

{noformat}
[root@cassandra2 ~]# lsof -u cassandra|wc -l
175416
[root@cassandra2 ~]# lsof -u cassandra|grep -c OpsCenter
174474
{noformat}

 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
 Attachments: c_file-descriptors_strace.tbz, cassandra_jstack.txt, 
 leak.log, position_hints.tgz, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes
 Max data size unlimitedunlimitedbytes
 Max stack size10485760 unlimitedbytes
 Max core file size00bytes
 Max resident set  unlimitedunlimitedbytes
 Max processes 1024 unlimitedprocesses
 Max open files4096 4096 files
 Max locked memory unlimitedunlimitedbytes
 Max address space unlimitedunlimitedbytes
 Max file locksunlimitedunlimitedlocks
 Max pending signals   1463314633signals  
 Max msgqueue size 819200   819200   bytes
 Max nice priority 00   
 Max realtime priority 00   
 Max realtime timeout  unlimitedunlimitedus 
 {noformat}
 Looks like the problem is not in limits.
 Before load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 166
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 164
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 180
 {noformat}
 After load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 967
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 1766
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 2578
 {noformat}
 Most opened files have names like:
 {noformat}
 java  16890 cassandra 1636r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1637r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1638r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1639r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1640r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-18 Thread graham sanderson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13825892#comment-13825892
 ] 

graham sanderson edited comment on CASSANDRA-6275 at 11/18/13 10:53 PM:


Note also, that most if not all of the deleted files are of the form

{code}
java14018 cassandra  586r   REG   8,33   8792499   1251 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4656-Data.db 
(deleted)
java14018 cassandra  587r   REG   8,33  27303760   1254 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4655-Data.db 
(deleted)
java14018 cassandra  588r   REG   8,33   8792499   1251 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4656-Data.db 
(deleted)
java14018 cassandra  589r   REG   8,33  27303760   1254 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4655-Data.db 
(deleted)
java14018 cassandra  590r   REG   8,33  10507214936 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4657-Data.db 
(deleted)
{code}
We have 7 data disks per node (don't know if this contributes to the problem), 
and the number of such (open but) deleted files is very ill balanced with 93% 
on two of the 7 disks (on this particular node)... the distribution of live 
data file size for OpsCenter/rollups60 is a little uneven with the same data 
mounts that have more deleted files having more actual live data, but the 
deleted file counts per mount point vary by several order of magnitudes whereas 
the data size itself does not.


was (Author: graham sanderson):
Note also, that most if not all of the deleted files are of the form

{code}
java14018 cassandra  586r   REG   8,33   8792499   1251 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4656-Data.db 
(deleted)
java14018 cassandra  587r   REG   8,33  27303760   1254 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4655-Data.db 
(deleted)
java14018 cassandra  588r   REG   8,33   8792499   1251 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4656-Data.db 
(deleted)
java14018 cassandra  589r   REG   8,33  27303760   1254 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4655-Data.db 
(deleted)
java14018 cassandra  590r   REG   8,33  10507214936 
/data/1/cassandra/OpsCenter/rollups60/OpsCenter-rollups60-jb-4657-Data.db 
(deleted)
{code}
We have 7 data disks (don't know if this contributes to the problem), and the 
number of such deleted files is very ill balanced with 93% on two of the 7 
disks (on this particular node)... the distribution of live data file size for 
OpsCenter/rollups60 is a little uneven with the same data mounts that have more 
deleted (but open) files having more actual live data, but the deleted file 
counts per mount point vary by several order of magnitudes whereas the data 
itself does not.

 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
 Attachments: c_file-descriptors_strace.tbz, cassandra_jstack.txt, 
 leak.log, position_hints.tgz, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes
 Max data size unlimitedunlimitedbytes
 Max stack size10485760 unlimitedbytes
 Max core file size00bytes
 Max resident set  unlimitedunlimitedbytes
 Max processes 1024 unlimitedprocesses
 Max open files4096 4096 files
 Max locked memory unlimitedunlimitedbytes
 Max address space unlimitedunlimitedbytes
 Max file locksunlimitedunlimitedlocks
 Max pending signals   1463314633signals  
 Max msgqueue size 

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-14 Thread Michael Shuler (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13823197#comment-13823197
 ] 

Michael Shuler edited comment on CASSANDRA-6275 at 11/15/13 1:39 AM:
-

c_file-descriptors_strace.tbz is a strace of the C* java process and children 
while running the query to about 27k open files.  This was on the cassandra-2.0 
branch in git.


was (Author: mshuler):
c_file-descriptors_strace.tbz is a strace of the C* java process and children 
while running the query to about 27k open files.

 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
Assignee: Marcus Eriksson
 Attachments: c_file-descriptors_strace.tbz, cassandra_jstack.txt, 
 leak.log, position_hints.tgz, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes
 Max data size unlimitedunlimitedbytes
 Max stack size10485760 unlimitedbytes
 Max core file size00bytes
 Max resident set  unlimitedunlimitedbytes
 Max processes 1024 unlimitedprocesses
 Max open files4096 4096 files
 Max locked memory unlimitedunlimitedbytes
 Max address space unlimitedunlimitedbytes
 Max file locksunlimitedunlimitedlocks
 Max pending signals   1463314633signals  
 Max msgqueue size 819200   819200   bytes
 Max nice priority 00   
 Max realtime priority 00   
 Max realtime timeout  unlimitedunlimitedus 
 {noformat}
 Looks like the problem is not in limits.
 Before load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 166
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 164
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 180
 {noformat}
 After load test:
 {noformat}
 cassandra-test0 ~]$ lsof -n | grep java | wc -l
 967
 cassandra-test1 ~]$ lsof -n | grep java | wc -l
 1766
 cassandra-test2 ~]$ lsof -n | grep java | wc -l
 2578
 {noformat}
 Most opened files have names like:
 {noformat}
 java  16890 cassandra 1636r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1637r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1638r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1639r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1640r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1641r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1642r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1643r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1644r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1645r  REG 202,17 161158485 
 655420 /var/lib/cassandra/data/system/paxos/system-paxos-jb-255-Data.db
 java  16890 cassandra 1646r  REG 202,17  88724987 
 655520 /var/lib/cassandra/data/system/paxos/system-paxos-jb-644-Data.db
 java  16890 cassandra 1647r  

[jira] [Comment Edited] (CASSANDRA-6275) 2.0.x leaks file handles

2013-11-11 Thread Gianluca Borello (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13819144#comment-13819144
 ] 

Gianluca Borello edited comment on CASSANDRA-6275 at 11/11/13 5:35 PM:
---

We are experiencing a similar issue in 2.0.2.

It started happening after we set a TTL for all our columns in a very limited 
datastore (just a few GBs).

We can easily see the fd count rapidly increase to 10+, and the majority of 
fds are (from lsof):

{noformat}
java  13168   cassandra  267r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  268r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  269r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  270r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  271r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  272r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  273r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
{noformat}

I'm attaching the log of the exception (leak.log). You can see the exceptions, 
and then Cassandra eventually shuts down. We had to temporarily downgrade to 
1.2.11



was (Author: gianlucaborello):
We are experiencing a similar issue in 2.0.2.

It started happening after we set a TTL for all our columns in a very limited 
datastore (just a few GBs).

We can easily see the fd count rapidly increase to 10+, and the majority of 
fds are (from lsof):

{noformat}
java  13168   cassandra  267r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  268r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  269r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  270r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  271r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  272r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
java  13168   cassandra  273r  REG9,0   273129  
671089723 
/raid0/cassandra/data/draios/process_counters_by_exe/draios-process_counters_by_exe-jb-231-Data.db
 (deleted)
{noformat}

I'm attaching the log of the exception. You can see the exceptions, and then 
Cassandra eventually shuts down. We had to temporarily downgrade to 1.2.11


 2.0.x leaks file handles
 

 Key: CASSANDRA-6275
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6275
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: java version 1.7.0_25
 Java(TM) SE Runtime Environment (build 1.7.0_25-b15)
 Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode)
 Linux cassandra-test1 2.6.32-279.el6.x86_64 #1 SMP Thu Jun 21 15:00:18 EDT 
 2012 x86_64 x86_64 x86_64 GNU/Linux
Reporter: Mikhail Mazursky
 Attachments: cassandra_jstack.txt, leak.log, slog.gz


 Looks like C* is leaking file descriptors when doing lots of CAS operations.
 {noformat}
 $ sudo cat /proc/15455/limits
 Limit Soft Limit   Hard Limit   Units
 Max cpu time  unlimitedunlimitedseconds  
 Max file size unlimitedunlimitedbytes