[
https://issues.apache.org/jira/browse/CASSANDRA-6651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13996397#comment-13996397
]
nayden kolev commented on CASSANDRA-6651:
-----------------------------------------
I also run into this on a 2.0.5.24 cluster. It seems one out of every 3-5
repairs hang for me. I have a couple of keyspaces and they are relatively small
- a few hundred MB for one, and a couple of GB for the other. Normally the
repair takes about an hour, if it does not hang. I have waited as long as a
couple of days for a hung one and it never completes - it stays in nodetool
tpstats output as 1 Active and 1 Pending in the AntiEntropySessions until the
node is restarted.
> Repair hanging
> --------------
>
> Key: CASSANDRA-6651
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6651
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Eitan Eibschutz
> Assignee: Yuki Morishita
>
> Hi,
> We have a 12 node cluster in PROD environment and we've noticed that repairs
> are never finishing. The behavior that we've observed is that a repair
> process will run until at some point it hangs and no other processing is
> happening.
> For example, at the moment, I have a repair process that has been running for
> two days and not finishing:
> nodetool tpstats is showing 2 active and 2 pending AntiEntropySessions
> nodetool compactionstats is showing:
> pending tasks: 0
> Active compaction remaining time : n/a
> nodetools netstats is showing:
> Mode: NORMAL
> Not sending any streams.
> Read Repair Statistics:
> Attempted: 0
> Mismatch (Blocking): 142110
> Mismatch (Background): 0
> Pool Name Active Pending Completed
> Commands n/a 0 107589657
> Responses n/a 0 116430785
> The last entry that I see in the log is:
> INFO [AntiEntropySessions:18] 2014-02-03 04:01:39,145 RepairJob.java (line
> 116) [repair #ae78c6c0-8c2b-11e3-b950-c3b81a36bc9b] requesting merkle trees
> for MyCF (to [/x.x.x.x, /y.y.y.y, /z.z.z.z])
> The repair started at 4am so it stopped after 1:40 minute.
> On node y.y.y.y I can see this in the log:
> INFO [MiscStage:1] 2014-02-03 04:01:38,985 ColumnFamilyStore.java (line 740)
> Enqueuing flush of Memtable-MyCF@1290890489(2176/5931 serialized/live bytes,
> 32 ops)
> INFO [FlushWriter:411] 2014-02-03 04:01:38,986 Memtable.java (line 333)
> Writing Memtable-MyCF@1290890489(2176/5931 serialized/live bytes, 32 ops)
> INFO [FlushWriter:411] 2014-02-03 04:01:39,048 Memtable.java (line 373)
> Completed flushing
> /var/lib/cassandra/main-db/data/MyKS/MyCF/MyKS-MyCF-jb-518-Data.db (1789
> bytes) for commitlog position ReplayPosition(segmentId=1390437013339,
> position=21868792)
> INFO [ScheduledTasks:1] 2014-02-03 05:00:04,794 ColumnFamilyStore.java (line
> 740) Enqueuing flush of Memtable-compaction_history@1649414699(1635/17360
> serialized/live bytes, 42 ops)
> So for some reason the merkle tree for this CF is never sent back to the node
> being repaired and it's hanging.
> I've also noticed that sometimes, restarting node y.y.y.y will cause the
> repair to resume.
> Another observation is that sometimes when restarting y.y.y.y it will not
> start with these errors:
> ERROR 16:34:18,485 Exception encountered during startup
> java.lang.IllegalStateException: Unfinished compactions reference missing
> sstables. This should never happen since compactions are marked finished
> before we start removing the old sstables.
> at
> org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:495)
> at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:264)
> at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:461)
> at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:504)
> java.lang.IllegalStateException: Unfinished compactions reference missing
> sstables. This should never happen since compactions are marked finished
> before we start removing the old sstables.
> at
> org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:495)
> at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:264)
> at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:461)
> at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:504)
> Exception encountered during startup: Unfinished compactions reference
> missing sstables. This should never happen since compactions are marked
> finished before we start removing the old sstables.
> And it will only restart after manually cleaning the compactions_in-progress
> folder.
> I'm not sure if these two issues are related but we've seen both on all the
> nodes in our cluster.
> I'll be happy to provide more info if needed as we are not sure what could
> cause this behavior.
> Another thing in our environment is that some of the Cassandra nodes have
> more than one network interface and RPC is listening on 0.0.0.0, not sure if
> it has anything to do with this.
> Thanks,
> Eitan
--
This message was sent by Atlassian JIRA
(v6.2#6252)