[ 
https://issues.apache.org/jira/browse/CASSANDRA-6651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13892345#comment-13892345
 ] 

Thunder Stumpges commented on CASSANDRA-6651:
---------------------------------------------

FWIW we have this exact same issue. We are running 2.0.3 on a 3 node cluster. 
It has happened multiple times, and happens more times than not when running 
nodetool repair. There is nearly always one or more AntiEntropySessions 
remaining according to tpstats.

One strange thing about the behavior I see is that the output of nodetool 
compactionstats returns 0 active compactions, yet when restarting, we get the 
exception about "Unfinished compactions reference missing sstables." It does 
seem like these two issues are related.

Another thing I see sometimes in the ouput from nodetool repair is the 
following message:
[2014-02-04 14:07:30,858] Starting repair command #7, repairing 768 ranges for 
keyspace thunder_test
[2014-02-04 14:08:30,862] Lost notification. You should check server log for 
repair status of keyspace thunder_test
[2014-02-04 14:08:30,870] Starting repair command #8, repairing 768 ranges for 
keyspace doan_synset
[2014-02-04 14:09:30,874] Lost notification. You should check server log for 
repair status of keyspace doan_synset

When this happens, it starts the next repair session immediately rather than 
waiting for the current one to finish. This doesn't however seem to always 
correlate to a hung session.

My logs don't look much/any different from the OP, but I'd be glad to provide 
any more details that might be helpful. We will be upgrading to 2.0.4 in the 
next couple days and I will report back if we see any difference in behavior.


> Repair hanging
> --------------
>
>                 Key: CASSANDRA-6651
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6651
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Eitan Eibschutz
>             Fix For: 2.0.3
>
>
> 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.1.5#6160)

Reply via email to