On Mon, Nov 15, 2010 at 1:03 PM, Reverend Chip <rev.c...@gmail.com> wrote: > I find X.21's data disk is full. "nodetool ring" says that X.21 has a > load of only 326.2 GB, but the 1T partition is full.
Load only tracks live data -- is the rest tmp files. > Somehow repair decided it needed to triple the data usage. I would like > to understand this, and I invite recovery suggestions. We have https://issues.apache.org/jira/browse/CASSANDRA-1674 open for repair space usage. > Meanwhile, I find this interesting sequence on X.20. > It could indicate compaction interfered with repair. Note presence of > TestAttrs-e-332 in the compaction: > > INFO 10:56:00,283 Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-332-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-351-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-368-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-386-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-401-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/Attrs/TestAttrs-e-414-Data.db')] > ... > INFO 01:26:56,175 Deleted /var/lib/cassandra/data/Attrs/TestAttrs-e-332-<> > ... > INFO 22:42:10,032 Need to re-stream file > /var/lib/cassandra/data/Attrs/TestAttrs-e-332-Data.db to /X.21 > ERROR 22:42:16,822 Error in ThreadPoolExecutor > java.lang.RuntimeException: java.io.IOException: Broken pipe > It looks like compaction and repair interfere with each other, and > compactions and repairs should stay out of each others' way. It looks to me like the sequence of events is: 1. streaming had a FD for -332 2. compaction deleted the file 3. X.21 ran out of space 4. the stream errored out 5. retrying the stream fails repeatedly Notably if one thread has a file open on linux and another thread/process deletes it, nothing bad happens and the file is unlinked when closed. (However attempting to do this causes errors on Windows, so we try to avoid that.) IOW I think the compaction/repair race is a bug, but not a smoking gun in this case unless I am wrong and 4. comes before 3. > It also looks like streaming doesn't recover gracefully from file deletion. If 5. comes before 3. then yes, otherwise I think erroring from out of disk space is the most likely culprit. -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com