David; 
all the packaged installations use the /var/lib/cassandra directory. Could you 
check your yaml config files and see if you are using this default directory  
for backups 
May want to change it to a location with more disk space. 


hope this helpsJan/  


     On Monday, March 16, 2015 2:52 PM, David Wahler <dwah...@indeed.com> wrote:
   

 We have a 16-node, globally-distributed cluster. running Cassandra
2.0.12. We're using the Datastax packages on CentOS 6.5.

Even though the total amount of data on each server is only a few
hundred MB (as measured by both du and the "load" metric), we're
seeing a problem where the disk usage is steadily increasing and
eventually filling up the 10GB /var/lib/cassandra partition. Running
"lsof" on the Cassandra process shows that it has open file handles
for thousands of deleted snapshot files:

$ sudo lsof -p 4753 | grep DEL -c
13314
$ sudo lsof -p 4753 | grep DEL | head
java 4753 cassandra DEL REG 253,6 538873
/var/lib/cassandra/data/keyspace/cf/snapshots/65bc8170-cc20-11e4-a355-0d37e54cc22e/keyspace-cf-jb-3979-Index.db
java 4753 cassandra DEL REG 253,6 538899
/var/lib/cassandra/data/keyspace/cf/snapshots/8cb41770-cc20-11e4-a355-0d37e54cc22e/keyspace-cf-jb-3983-Index.db
...etc...

We're not manually creating these snapshots; they're being generated
by periodic runs of "nodetool repair -pr". There are some errors in
system.log that seem to be related:

ERROR [RepairJobTask:10] 2015-03-16 02:02:12,485 RepairJob.java (line
143) Error occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /10.1.1.188
        at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:81)
        at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:344)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
ERROR [AntiEntropySessions:4] 2015-03-16 02:02:12,486
RepairSession.java (line 288) [repair
#55a8eb50-cbaa-11e4-9af9-27d7677e5965] session completed with the
following error
java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:323)
        at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:144)
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
ERROR [AntiEntropySessions:4] 2015-03-16 02:02:12,488
CassandraDaemon.java (line 199) Exception in thread
Thread[AntiEntropySessions:4,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during
snapshot creation.
        at com.google.common.base.Throwables.propagate(Throwables.java:160)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:323)
        at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:144)
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
        ... 3 more

Has anyone encountered this problem before? The same stack trace shows
up in CASSANDRA-8020, but that bug was supposedly introduced in 2.1.0
and fixed in 2.1.1. In any case, we don't want to upgrade to 2.1.x,
since the consensus on this list seems to be that it's not yet
production-ready.

I'm fairly new to Cassandra, so general troubleshooting tips would
also be much appreciated.

Thanks,
-- David


  

Reply via email to