You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jan Karlsson (JIRA)" <ji...@apache.org> on 2016/02/08 15:32:39 UTC

[jira] [Commented] (CASSANDRA-8643) merkle tree creation fails with NoSuchElementException

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

Jan Karlsson commented on CASSANDRA-8643:
-----------------------------------------

We hit it again. This time we had more time debugging the situation and we might have found the problem. It started occuring when we switched to LeveledCompactionStrategy. However it does not occur consistently. We usually get it once every 2-3 runs.

We enabled assertions and got "received out of order wrt". The problem we found is that the ranges of the tables are intersecting but the getScanners method in LCS expects them to be non-intersecting (as all sstables in the same level should not be intersecting). 

It could be that during the snapshot, a compaction occurs which writes more sstables into the level. Then when it is supplied to the repair job, it fails due to the ranges intersecting in the new and old sstables. 

When we tried repairing with -par, we did not hit it. It also worked with 2.2.4 (which runs -par by default).

> merkle tree creation fails with NoSuchElementException
> ------------------------------------------------------
>
>                 Key: CASSANDRA-8643
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8643
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: We are running on a three node cluster with three in replication(C* 2.1.1). It uses a default C* installation and STCS.
>            Reporter: Jan Karlsson
>             Fix For: 2.1.3
>
>
> We have a problem that we encountered during testing over the weekend. 
> During the tests we noticed that repairs started to fail. This error has occured on multiple non-coordinator nodes during repair. It also ran at least once without producing this error.
> We run repair -pr on all nodes on different days. CPU values were around 40% and disk was 50% full.
> From what I understand, the coordinator asked for merkle trees from the other two nodes. However one of the nodes fails to create his merkle tree.
> Unfortunately we do not have a way to reproduce this problem.
> The coordinator receives:
> {noformat}
> 2015-01-09T17:55:57.091+0100  INFO [RepairJobTask:4] RepairJob.java:145 [repair #59455950-9820-11e4-b5c1-7797064e1316] requesting merkle trees for censored (to [/xx.90, /xx.98, /xx.82])
> 2015-01-09T17:55:58.516+0100  INFO [AntiEntropyStage:1] RepairSession.java:171 [repair #59455950-9820-11e4-b5c1-7797064e1316] Received merkle tree for censored from /xx.90
> 2015-01-09T17:55:59.581+0100 ERROR [AntiEntropySessions:76] RepairSession.java:303 [repair #59455950-9820-11e4-b5c1-7797064e1316] session completed with the following error
> org.apache.cassandra.exceptions.RepairException: [repair #59455950-9820-11e4-b5c1-7797064e1316 on censored/censored, (-6476420463551243930,-6471459119674373580]] Validation failed in /xx.98
>         at org.apache.cassandra.repair.RepairSession.validationComplete(RepairSession.java:166) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:384) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:126) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_51]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
> 2015-01-09T17:55:59.582+0100 ERROR [AntiEntropySessions:76] CassandraDaemon.java:153 Exception in thread Thread[AntiEntropySessions:76,5,RMI Runtime]
> java.lang.RuntimeException: org.apache.cassandra.exceptions.RepairException: [repair #59455950-9820-11e4-b5c1-7797064e1316 on censored/censored, (-6476420463551243930,-6471459119674373580]] Validation failed in /xx.98
>         at com.google.common.base.Throwables.propagate(Throwables.java:160) ~[guava-16.0.jar:na]
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_51]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_51]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
>        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51] Caused by: org.apache.cassandra.exceptions.RepairException: [repair #59455950-9820-11e4-b5c1-7797064e1316 on censored/censored, (-6476420463551243930,-6471459119674373580]] Validation failed in /xx.98
>         at org.apache.cassandra.repair.RepairSession.validationComplete(RepairSession.java:166) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:384) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:126) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         ... 3 common frames omitted
> {noformat}
> While one of the other nodes produces this error:
> {noformat}
> 2015-01-09T17:55:59.574+0100 ERROR [ValidationExecutor:16] Validator.java:232 Failed creating a merkle tree for [repair #59455950-9820-11e4-b5c1-7797064e1316 on censored/censored, (-6476420463551243930,-6471459119674373580]], /xx.82 (see log for details)
> 2015-01-09T17:55:59.578+0100 ERROR [ValidationExecutor:16] CassandraDaemon.java:153 Exception in thread Thread[ValidationExecutor:16,1,main]
> java.util.NoSuchElementException: null
>         at com.google.common.collect.AbstractIterator.next(AbstractIterator.java:154) ~[guava-16.0.jar:na]
>         at org.apache.cassandra.repair.Validator.add(Validator.java:137) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:930) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:97) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:557) ~[apache-cassandra-2.1.1.jar:2.1.1]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_51]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)