Basically I tweaked the phi, put in more verbose GC reporting and decided to do a compaction before I proceed. I'm getting this on the node where compaction is being run. And the system log for the other two nodes follows. It's obvious that the cluster is sick, but I can't determine why -- there are no overwhelming GC evidence as far as I can see. I didn't start compaction on node #3, somehow
it attempts to do it anyhow.

===============================================
Node #2 (compaction is being run):

INFO [CompactionExecutor:2] 2011-12-05 14:19:36,741 CompactionManager.java (line 608) Compacted to /data/cassandra_data/data/system/LocationInfo-tmp-g-72-Data.db. 967 to 561 (~58% of
original) bytes for 4 keys.  Time: 71ms.
INFO [main] 2011-12-05 14:19:36,941 Mx4jTool.java (line 67) mx4j successfuly loaded INFO [GossipStage:1] 2011-12-05 14:19:36,943 Gossiper.java (line 715) Node /130.199.185.193 has restarted, now UP again INFO [GossipStage:1] 2011-12-05 14:19:36,943 Gossiper.java (line 683) InetAddress /130.199.185.193 is now UP INFO [GossipStage:1] 2011-12-05 14:19:36,971 StorageService.java (line 819) Node /130.199.185.193 state jump to normal INFO [GossipStage:1] 2011-12-05 14:19:36,971 Gossiper.java (line 715) Node /130.199.185.195 has restarted, now UP again INFO [GossipStage:1] 2011-12-05 14:19:36,971 Gossiper.java (line 683) InetAddress /130.199.185.195 is now UP INFO [GossipStage:1] 2011-12-05 14:19:36,974 StorageService.java (line 819) Node /130.199.185.195 state jump to normal INFO [main] 2011-12-05 14:19:37,003 CassandraDaemon.java (line 115) Binding thrift service to cassandra02.usatlas.bnl.gov/130.199.185.194:9160 INFO [main] 2011-12-05 14:19:37,016 CassandraDaemon.java (line 124) Using TFastFramedTransport with a max frame size of 15728640 bytes. INFO [main] 2011-12-05 14:19:37,018 CassandraDaemon.java (line 151) Using synchronous/threadpool thrift server on cassandra02.usatlas.bnl.gov/130.199.185.194 : 9160 INFO [Thread-6] 2011-12-05 14:19:37,019 CassandraDaemon.java (line 203) Listening for thrift clients... INFO [GossipTasks:1] 2011-12-05 14:19:50,601 Gossiper.java (line 697) InetAddress /130.199.185.195 is now dead. ERROR [HintedHandoff:1] 2011-12-05 14:20:37,954 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.193 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.193 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 3 more
ERROR [HintedHandoff:1] 2011-12-05 14:20:37,956 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.193 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.193 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397)

=====================================================================
Node #1 (nothing run)


INFO [main] 2011-12-05 14:16:15,779 CassandraDaemon.java (line 115) Binding thrift service to cassandra01.usatlas.bnl.gov/130.199.185.193:9160 INFO [main] 2011-12-05 14:16:15,782 CassandraDaemon.java (line 124) Using TFastFramedTransport with a max frame size of 15728640 bytes. INFO [main] 2011-12-05 14:16:15,784 CassandraDaemon.java (line 151) Using synchronous/threadpool thrift server on cassandra01.usatlas.bnl.gov/130.199.185.193 : 9160 INFO [Thread-5] 2011-12-05 14:16:15,785 CassandraDaemon.java (line 203) Listening for thrift clients... INFO [HintedHandoff:1] 2011-12-05 14:16:50,578 HintedHandOffManager.java (line 323) Started hinted handoff for endpoint /130.199.185.195 INFO [ScheduledTasks:1] 2011-12-05 14:16:53,220 GCInspector.java (line 122) GC for ParNew: 284 ms for 1 collections, 1434456744 used; max is 8178892800 INFO [HintedHandoff:1] 2011-12-05 14:16:56,166 HintedHandOffManager.java (line 379) Finished hinted handoff of 0 rows to endpoint /130.199.185.195 INFO [GossipTasks:1] 2011-12-05 14:17:19,310 Gossiper.java (line 697) InetAddress /130.199.185.194 is now dead. INFO [GossipTasks:1] 2011-12-05 14:17:50,326 Gossiper.java (line 697) InetAddress /130.199.185.195 is now dead. ERROR [HintedHandoff:1] 2011-12-05 14:17:57,173 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 3 more
ERROR [HintedHandoff:1] 2011-12-05 14:17:57,175 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 3 more
INFO [GossipStage:1] 2011-12-05 14:19:36,938 Gossiper.java (line 715) Node /130.199.185.194 has restarted, now UP again INFO [GossipStage:1] 2011-12-05 14:19:36,938 Gossiper.java (line 683) InetAddress /130.199.185.194 is now UP INFO [GossipStage:1] 2011-12-05 14:19:36,939 StorageService.java (line 819) Node /130.199.185.194 state jump to normal ERROR [HintedHandoff:2] 2011-12-05 14:20:37,944 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:2,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms

========================================================================================

Node #3 (nothing run):

INFO [GossipStage:1] 2011-12-05 14:23:17,762 StorageService.java (line 819) Node /130.199.185.193 state jump to normal INFO [GossipStage:1] 2011-12-05 14:23:17,762 Gossiper.java (line 715) Node /130.199.185.194 has restarted, now UP again INFO [GossipStage:1] 2011-12-05 14:23:17,763 Gossiper.java (line 683) InetAddress /130.199.185.194 is now UP INFO [GossipStage:1] 2011-12-05 14:23:18,045 StorageService.java (line 819) Node /130.199.185.194 state jump to normal INFO [CompactionExecutor:2] 2011-12-05 14:23:18,055 CompactionManager.java (line 608) Compacted to /data/cassandra_data/data/system/LocationInfo-tmp-g-32-Data.db. 967 to 561 (~58% of origina
l) bytes for 4 keys.  Time: 323ms.
INFO [main] 2011-12-05 14:23:18,074 CassandraDaemon.java (line 115) Binding thrift service to cassandra03.usatlas.bnl.gov/130.199.185.195:9160 INFO [main] 2011-12-05 14:23:18,077 CassandraDaemon.java (line 124) Using TFastFramedTransport with a max frame size of 15728640 bytes. INFO [main] 2011-12-05 14:23:18,080 CassandraDaemon.java (line 151) Using synchronous/threadpool thrift server on cassandra03.usatlas.bnl.gov/130.199.185.195 : 9160 INFO [Thread-6] 2011-12-05 14:23:18,080 CassandraDaemon.java (line 203) Listening for thrift clients... INFO [HintedHandoff:1] 2011-12-05 14:23:24,535 HintedHandOffManager.java (line 323) Started hinted handoff for endpoint /130.199.185.193 INFO [HintedHandoff:1] 2011-12-05 14:23:24,537 HintedHandOffManager.java (line 379) Finished hinted handoff of 0 rows to endpoint /130.199.185.193 ERROR [HintedHandoff:1] 2011-12-05 14:24:25,553 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 3 more
ERROR [HintedHandoff:1] 2011-12-05 14:24:25,555 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.RuntimeException: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.RuntimeException: Could not reach schema agreement with /130.199.185.194 in 60000ms at org.apache.cassandra.db.HintedHandOffManager.waitForSchemaAgreement(HintedHandOffManager.java:293) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:304) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 3 more
INFO [CompactionExecutor:5] 2011-12-05 14:27:11,220 CompactionManager.java (line 494) Nothing to compact in IndexInfo; use forceUserDefinedCompaction if you wish to force compaction of single
 sstables (e.g. for tombstone collection)
INFO [CompactionExecutor:6] 2011-12-05 14:27:11,221 CompactionManager.java (line 494) Nothing to compact in LocationInfo; use forceUserDefinedCompaction if you wish to force compaction of sin
gle sstables (e.g. for tombstone collection)
INFO [CompactionExecutor:7] 2011-12-05 14:27:11,222 CompactionManager.java (line 542) Compacting Major: [SSTableReader(path='/data/cassandra_data/data/system/Migrations-g-72-Data.db'), SSTabl
eReader(path='/data/cassandra_data/data/system/Migrations-g-71-Data.db')]
INFO [CompactionExecutor:7] 2011-12-05 14:27:11,258 CompactionManager.java (line 608) Compacted to /data/cassandra_data/data/system/Migrations-tmp-g-73-Data.db. 398,726 to 398,662 (~99% of o
riginal) bytes for 1 keys.  Time: 36ms.
INFO [CompactionExecutor:10] 2011-12-05 14:27:11,260 CompactionManager.java (line 542) Compacting Major: [SSTableReader(path='/data/cassandra_data/data/system/Schema-g-72-Data.db'), SSTableRe
ader(path='/data/cassandra_data/data/system/Schema-g-71-Data.db')]
INFO [CompactionExecutor:10] 2011-12-05 14:27:11,291 CompactionManager.java (line 608) Compacted to /data/cassandra_data/data/system/Schema-tmp-g-73-Data.db. 175,627 to 175,518 (~99% of orig
inal) bytes for 54 keys.  Time: 30ms.
INFO [CompactionExecutor:12] 2011-12-05 14:27:11,293 CompactionManager.java (line 494) Nothing to compact in indexRegistry; use forceUserDefinedCompaction if you wish to force compaction of s
ingle sstables (e.g. for tombstone collection)
INFO [CompactionExecutor:13] 2011-12-05 14:27:11,295 StorageService.java (line 2228) requesting GC to free disk space WARN [CompactionExecutor:13] 2011-12-05 14:27:34,920 CompactionManager.java (line 509) insufficient space to compact all requested files SSTableReader(path='/data/cassandra_data/data/PANDA/jo bs-g-54-Data.db'), SSTableReader(path='/data/cassandra_data/data/PANDA/jobs-g-55-Data.db') ERROR [CompactionExecutor:13] 2011-12-05 14:27:34,920 CompactionManager.java (line 513) insufficient space to compact even the two smallest files, aborting


On 12/4/2011 7:17 PM, Peter Schuller wrote:
As a side effect of the failed repair (so it seems) the disk usage on the
affected node prevents compaction from working. It still works on
the remaining nodes (we have 3 total).
Is there a way to scrub the extraneous data?
This is one of the reasons why killing an in-process repair is a bad thing :(

If you do not have enough disk space for any kind of compaction to
work, then no, unfortunately there is no easy way to get rid of the
data.

You can go to extra trouble such as moving the entire node to some
other machine (e.g. firewalled from the cluster) with more disk and
run compaction there and then "move it back", but that is kind of
painful to do. Another option is to decommission the node and replace
it. However, be aware that (1) that leaves the ring with less capacity
for a while, and (2) when you decommission, the data you stream from
that node to others would be artificially inflated due to the repair
so there is some risk of "infecting" the other nodes with a large data
set.

I should mention that if you have no traffic running against the
cluster, one way is to just remove all the data and then run repair
afterwards. But that implies that you're trusting that (1) no reads
are going to the cluster (else you might serve reads based on missing
data) and (2) that you are comfortable with loss of the data on the
node. (2) might be okay if you're e.g. writing at QUORUM at all times
and have RF>= 3 (basically, this is as if the node would have been
lost due to hardware breakage).

A faster way to reconstruct the node would be to delete the data from
your keyspaces (except the system keyspace), start the node (now
missing data), and run 'nodetool rebuild' after
https://issues.apache.org/jira/browse/CASSANDRA-3483 is done. The
patch attached to that ticket should work for 0.8.6 I suspect (but no
guarantees). This also assumes you have no reads running against the
cluster.


Reply via email to