[
https://issues.apache.org/jira/browse/CASSANDRA-12860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Bing Wu updated CASSANDRA-12860:
--------------------------------
Environment: CentOS 6.7, Java HotSpot(TM) 64-Bit Server VM (build
25.102-b14, mixed mode), Cassandra 3.5.0, fresh install (was: CentOS 7, JDK
8u60, Cassandra 2.2.2 (upgraded from 2.1.5))
Priority: Critical (was: Minor)
Description:
Summary of symptom:
- Set up is a multi-region cluster in AWS (5 regions). Each region has at least
4 hosts with RF=1/2 number of nodes, using V-nodes (256)
- How to reproduce:
-- On node A, start this repair job (again we are running fresh 3.5.0): {code}
sudo nodetool repair -pr my_keyspace > /tmp/repair.log 2>&1 & {code}
-- Job starts fine, reporting progress like {noformat} [2016-10-28
21:57:44,427] Repair session 03b6ca61-9d59-11e6-b118-b9abfef3117a for range
[(2427717901143689479,2428773541412139342]] finished (progress: 30%){noformat}
-- Then manually shutdown another node (node B) in the same region (haven't
tried with other region yet but expect the same behavior from past experience)
-- Shortly after that seeing this message from job log (as well as in
system.log) on node A: {noformat} [2016-10-28 21:59:40,835] Repair session
04000861-9d59-11e6-b118-b9abfef3117a for range
[(6981391007853361210,6983870256023436902]] failed with error Endpoint
/52.220.127.177 died (progress: 59%)
{noformat}
-- At this point, repair job seems to hang:
--- no further messages from job log
--- nor any related messages in system.log
--- CPU stayed low (<5%)
-- After an hour (1hr), manually kill the repair jobs using "ps -eaf | grep
repair"
-- Restart C* on node A
--- Verified system is up and no error messages in system.log
--- Also verified that there is no error messages from node B
-- After node A settles down (e.g. no new messages from system.log), restart
the same repair job: {code} sudo nodetool repair -pr my_keyspace >
/tmp/repair.log 2>&1 & {code}
-- Job failes pretty quickly, reporting error from another node K: {noformat}
<production>[[email protected] ~]$ tail -f /tmp/repair.log
nohup: ignoring input
[2016-10-28 22:15:31,770] Starting repair command #1, repairing keyspace
my_keyspace with repair options (parallelism: parallel, primary range: true,
incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [], hosts:
[], # of ranges: 256)
[2016-10-28 22:15:55,375] Repair session 17b7c390-9d5c-11e6-ba28-61f7d2732e5e
for range [(4029874034937227774,4033949979656106020]] failed with error [repair
#17b7c390-9d5c-11e6-ba28-61f7d2732e5e on my_keyspace/atable,
[(4029874034937227774,4033949979656106020]]] Validation failed in /NodeK
(progress: 1%) {noformat}
-- Go to node K and tail/view system.log, seeing: {noformat} ERROR
[ValidationExecutor:3] 2016-10-28 22:15:55,226 CompactionManager.java:1320 -
Cannot start multiple repair sessions over the same sstables
ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,226 Validator.java:261 -
Failed creating a merkle tree for [repair #17b7c390-9d5c-11e6-ba28-61f7d2732e5e
on my_keyspace/atable, [(4029874034937227774,4033949979656106020]]],
/52.220.127.190 (see log for details)
ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,227 CassandraDaemon.java:195 -
Exception in thread Thread[ValidationExecutor:3,1,main]
java.lang.RuntimeException: Cannot start multiple repair sessions over the same
sstables
at
org.apache.cassandra.db.compaction.CompactionManager.getSSTablesToValidate(CompactionManager.java:1321)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1211)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:81)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager$11.call(CompactionManager.java:841)
~[apache-cassandra-3.5.0.jar:3.5.0]
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
~[na:1.8.0_102]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_102]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_102]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,468
CompactionManager.java:1320 - Cannot start multiple repair sessions over the
same sstables
ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,468 Validator.java:261 -
Failed creating a merkle tree for [repair #17b7c390-9d5c-11e6-ba28-61f7d2732e5e
on my_keyspace/btable, [(4029874034937227774,4033949979656106020]]],
/52.220.127.190 (see log for details)
ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,469 CassandraDaemon.java:195 -
Exception in thread Thread[ValidationExecutor:3,1,main]
java.lang.RuntimeException: Cannot start multiple repair sessions over the same
sstables
at
org.apache.cassandra.db.compaction.CompactionManager.getSSTablesToValidate(CompactionManager.java:1321)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1211)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:81)
~[apache-cassandra-3.5.0.jar:3.5.0]
at
org.apache.cassandra.db.compaction.CompactionManager$11.call(CompactionManager.java:841)
~[apache-cassandra-3.5.0.jar:3.5.0]
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
~[na:1.8.0_102]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_102]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_102]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
{noformat}
-- At this point, we are back to where we were: we had to kill the repair job
on node A, then restart C* on node K, and seeing the same error (cannot
was:
Sometimes the repair fails:
{code}
ERROR [Repair#3:1] 2015-10-14 06:22:56,490 CassandraDaemon.java:185 - Exception
in thread Thread[Repair#3:1,5,RMI Runtime]
com.google.common.util.concurrent.UncheckedExecutionException:
org.apache.cassandra.exceptions.RepairException: [repair
#018adc70-723c-11e5-b0d8-6b2151e4d388 on keyspace/table,
(2414492737393085601,27880539413409
54029]] Validation failed in /w.y.x.z
at
com.google.common.util.concurrent.Futures.wrapAndThrowUnchecked(Futures.java:1387)
~[guava-16.0.jar:na]
at
com.google.common.util.concurrent.Futures.getUnchecked(Futures.java:1373)
~[guava-16.0.jar:na]
at org.apache.cassandra.repair.RepairJob.run(RepairJob.java:169)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_60]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
~[na:1.8.0_60]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_60]
Caused by: org.apache.cassandra.exceptions.RepairException: [repair
#018adc70-723c-11e5-b0d8-6b2151e4d388 on keyspace/table,
(2414492737393085601,2788053941340954029]] Validation failed in /w.y.x.z
at
org.apache.cassandra.repair.ValidationTask.treeReceived(ValidationTask.java:64)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.repair.RepairSession.validationComplete(RepairSession.java:183)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:399)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:163)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:66)
~[apache-cassandra-2.2.2.jar:2.2.2]
... 3 common frames omitted
{code}
And here is the w.y.x.z side:
{code}
ERROR [ValidationExecutor:7] 2015-10-14 06:22:56,487
CompactionManager.java:1053 - Cannot start multiple repair sessions over the
same sstables
ERROR [ValidationExecutor:7] 2015-10-14 06:22:56,487 Validator.java:246 -
Failed creating a merkle tree for [repair #018adc70-723c-11e5-b0d8-6b2151e4d388
on keyspace/table, (2414492737393085601,2788053941340954029]], /a.b.c.d (see
log for details)
ERROR [ValidationExecutor:7] 2015-10-14 06:22:56,488 CassandraDaemon.java:185 -
Exception in thread Thread[ValidationExecutor:7,1,main]
java.lang.RuntimeException: Cannot start multiple repair sessions over the same
sstables
at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1054)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:86)
~[apache-cassandra-2.2.2.jar:2.2.2]
at
org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:652)
~[apache-cassandra-2.2.2.jar:2.2.2]
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
~[na:1.8.0_60]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_60]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_60]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60]
...
ERROR [Reference-Reaper:1] 2015-10-14 06:23:21,439 Ref.java:187 - LEAK
DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@74fc054a) to class
org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1949471967:/home/cassandra/dsc-cassandra-2.2.2/bin/../data/data/keyspace/table-b15521b0621111e4bbedcdee5e027297/la-1195-big
was not released before the reference was garbage collected
{code}
Summary: Nodetool repair fragile: cannot properly recover from single
node failure. Has to restart all nodes in order to repair again (was: CLONE -
RepairException: [repair #... on .../..., (...,...]] Validation failed in
/w.x.y.z)
> Nodetool repair fragile: cannot properly recover from single node failure.
> Has to restart all nodes in order to repair again
> ----------------------------------------------------------------------------------------------------------------------------
>
> Key: CASSANDRA-12860
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12860
> Project: Cassandra
> Issue Type: Bug
> Environment: CentOS 6.7, Java HotSpot(TM) 64-Bit Server VM (build
> 25.102-b14, mixed mode), Cassandra 3.5.0, fresh install
> Reporter: Bing Wu
> Priority: Critical
>
> Summary of symptom:
> - Set up is a multi-region cluster in AWS (5 regions). Each region has at
> least 4 hosts with RF=1/2 number of nodes, using V-nodes (256)
> - How to reproduce:
> -- On node A, start this repair job (again we are running fresh 3.5.0):
> {code} sudo nodetool repair -pr my_keyspace > /tmp/repair.log 2>&1 & {code}
> -- Job starts fine, reporting progress like {noformat} [2016-10-28
> 21:57:44,427] Repair session 03b6ca61-9d59-11e6-b118-b9abfef3117a for range
> [(2427717901143689479,2428773541412139342]] finished (progress: 30%){noformat}
> -- Then manually shutdown another node (node B) in the same region (haven't
> tried with other region yet but expect the same behavior from past experience)
> -- Shortly after that seeing this message from job log (as well as in
> system.log) on node A: {noformat} [2016-10-28 21:59:40,835] Repair session
> 04000861-9d59-11e6-b118-b9abfef3117a for range
> [(6981391007853361210,6983870256023436902]] failed with error Endpoint
> /52.220.127.177 died (progress: 59%)
> {noformat}
> -- At this point, repair job seems to hang:
> --- no further messages from job log
> --- nor any related messages in system.log
> --- CPU stayed low (<5%)
> -- After an hour (1hr), manually kill the repair jobs using "ps -eaf | grep
> repair"
> -- Restart C* on node A
> --- Verified system is up and no error messages in system.log
> --- Also verified that there is no error messages from node B
> -- After node A settles down (e.g. no new messages from system.log), restart
> the same repair job: {code} sudo nodetool repair -pr my_keyspace >
> /tmp/repair.log 2>&1 & {code}
> -- Job failes pretty quickly, reporting error from another node K: {noformat}
> <production>[[email protected] ~]$ tail -f
> /tmp/repair.log
> nohup: ignoring input
> [2016-10-28 22:15:31,770] Starting repair command #1, repairing keyspace
> my_keyspace with repair options (parallelism: parallel, primary range: true,
> incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [],
> hosts: [], # of ranges: 256)
> [2016-10-28 22:15:55,375] Repair session 17b7c390-9d5c-11e6-ba28-61f7d2732e5e
> for range [(4029874034937227774,4033949979656106020]] failed with error
> [repair #17b7c390-9d5c-11e6-ba28-61f7d2732e5e on my_keyspace/atable,
> [(4029874034937227774,4033949979656106020]]] Validation failed in /NodeK
> (progress: 1%) {noformat}
> -- Go to node K and tail/view system.log, seeing: {noformat} ERROR
> [ValidationExecutor:3] 2016-10-28 22:15:55,226 CompactionManager.java:1320 -
> Cannot start multiple repair sessions over the same sstables
> ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,226 Validator.java:261 -
> Failed creating a merkle tree for [repair
> #17b7c390-9d5c-11e6-ba28-61f7d2732e5e on my_keyspace/atable,
> [(4029874034937227774,4033949979656106020]]], /52.220.127.190 (see log for
> details)
> ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,227 CassandraDaemon.java:195
> - Exception in thread Thread[ValidationExecutor:3,1,main]
> java.lang.RuntimeException: Cannot start multiple repair sessions over the
> same sstables
> at
> org.apache.cassandra.db.compaction.CompactionManager.getSSTablesToValidate(CompactionManager.java:1321)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1211)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:81)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager$11.call(CompactionManager.java:841)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> ~[na:1.8.0_102]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> ~[na:1.8.0_102]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> [na:1.8.0_102]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
> ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,468
> CompactionManager.java:1320 - Cannot start multiple repair sessions over the
> same sstables
> ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,468 Validator.java:261 -
> Failed creating a merkle tree for [repair
> #17b7c390-9d5c-11e6-ba28-61f7d2732e5e on my_keyspace/btable,
> [(4029874034937227774,4033949979656106020]]], /52.220.127.190 (see log for
> details)
> ERROR [ValidationExecutor:3] 2016-10-28 22:15:55,469 CassandraDaemon.java:195
> - Exception in thread Thread[ValidationExecutor:3,1,main]
> java.lang.RuntimeException: Cannot start multiple repair sessions over the
> same sstables
> at
> org.apache.cassandra.db.compaction.CompactionManager.getSSTablesToValidate(CompactionManager.java:1321)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1211)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:81)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at
> org.apache.cassandra.db.compaction.CompactionManager$11.call(CompactionManager.java:841)
> ~[apache-cassandra-3.5.0.jar:3.5.0]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> ~[na:1.8.0_102]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> ~[na:1.8.0_102]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> [na:1.8.0_102]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
> {noformat}
> -- At this point, we are back to where we were: we had to kill the repair job
> on node A, then restart C* on node K, and seeing the same error (cannot
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)