[
https://issues.apache.org/jira/browse/CASSANDRA-7904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14132172#comment-14132172
]
Razi Khaja commented on CASSANDRA-7904:
---------------------------------------
I think this might be related since it mentions *RepairJob* ... I hope it is
helpful.
{code}
INFO [AntiEntropyStage:1] 2014-09-12 16:36:29,536 RepairSession.java (line
166) [repair #ec6b4340-3abd-11e4-b32d-db378a0ca7f3] Received merkle tree for
genome_protein_v10 from /XXX.XXX.XXX.XXX
ERROR [MiscStage:58] 2014-09-12 16:36:29,537 CassandraDaemon.java (line 199)
Exception in thread Thread[MiscStage:58,5,main]
java.lang.IllegalArgumentException: Unknown keyspace/cf pair
(megalink.probe_gene_v24)
at
org.apache.cassandra.db.Keyspace.getColumnFamilyStore(Keyspace.java:171)
at
org.apache.cassandra.service.SnapshotVerbHandler.doVerb(SnapshotVerbHandler.java:42)
at
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
ERROR [RepairJobTask:7] 2014-09-12 16:36:29,537 RepairJob.java (line 125) Error
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /XXX.XXX.XXX.XXX
at
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:81)
at
org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:47)
at
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
ERROR [AntiEntropySessions:73] 2014-09-12 16:36:29,540 RepairSession.java (line
288) [repair #ec6b4340-3abd-11e4-b32d-db378a0ca7f3] session completed with the
following error
java.io.IOException: Failed during snapshot creation.
at
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:323)
at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:126)
at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
ERROR [AntiEntropySessions:73] 2014-09-12 16:36:29,543 CassandraDaemon.java
(line 199) Exception in thread Thread[AntiEntropySessions:73,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot
creation.
at com.google.common.base.Throwables.propagate(Throwables.java:160)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.IOException: Failed during snapshot creation.
at
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:323)
at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:126)
at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
... 3 more
{code}
> Repair hangs
> ------------
>
> Key: CASSANDRA-7904
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7904
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: C* 2.0.10, ubuntu 14.04, Java HotSpot(TM) 64-Bit Server,
> java version "1.7.0_45"
> Reporter: Duncan Sands
> Attachments: ls-172.18.68.138, ls-192.168.21.13, ls-192.168.60.134,
> ls-192.168.60.136
>
>
> Cluster of 22 nodes spread over 4 data centres. Not used on the weekend, so
> repair is run on all nodes (in a staggered fashion) on the weekend. Nodetool
> options: -par -pr. There is usually some overlap in the repairs: repair on
> one node may well still be running when repair is started on the next node.
> Repair hangs for some of the nodes almost every weekend. It hung last
> weekend, here are the details:
> In the whole cluster, only one node had an exception since C* was last
> restarted. This node is 192.168.60.136 and the exception is harmless: a
> client disconnected abruptly.
> tpstats
> 4 nodes have a non-zero value for "active" or "pending" in
> AntiEntropySessions. These nodes all have Active => 1 and Pending => 1. The
> nodes are:
> 192.168.21.13 (data centre R)
> 192.168.60.134 (data centre A)
> 192.168.60.136 (data centre A)
> 172.18.68.138 (data centre Z)
> compactionstats:
> No compactions. All nodes have:
> pending tasks: 0
> Active compaction remaining time : n/a
> netstats:
> All except one node have nothing. One node (192.168.60.131, not one of the
> nodes listed in the tpstats section above) has (note the Responses Pending
> value of 1):
> Mode: NORMAL
> Not sending any streams.
> Read Repair Statistics:
> Attempted: 4233
> Mismatch (Blocking): 0
> Mismatch (Background): 243
> Pool Name Active Pending Completed
> Commands n/a 0 34785445
> Responses n/a 1 38567167
> Repair sessions
> I looked for repair sessions that failed to complete. On 3 of the 4 nodes
> mentioned in tpstats above I found that they had sent merkle tree requests
> and got responses from all but one node. In the log file for the node that
> failed to respond there is no sign that it ever received the request. On 1
> node (172.18.68.138) it looks like responses were received from every node,
> some streaming was done, and then... nothing. Details:
> Node 192.168.21.13 (data centre R):
> Sent merkle trees to /172.18.33.24, /192.168.60.140, /192.168.60.142,
> /172.18.68.139, /172.18.68.138, /172.18.33.22, /192.168.21.13 for table
> brokers, never got a response from /172.18.68.139. On /172.18.68.139, just
> before this time it sent a response for the same repair session but a
> different table, and there is no record of it receiving a request for table
> brokers.
> Node 192.168.60.134 (data centre A):
> Sent merkle trees to /172.18.68.139, /172.18.68.138, /192.168.60.132,
> /192.168.21.14, /192.168.60.134 for table swxess_outbound, never got a
> response from /172.18.68.138. On /172.18.68.138, just before this time it
> sent a response for the same repair session but a different table, and there
> is no record of it receiving a request for table swxess_outbound.
> Node 192.168.60.136 (data centre A):
> Sent merkle trees to /192.168.60.142, /172.18.68.139, /192.168.60.136 for
> table rollups7200, never got a response from /172.18.68.139. This repair
> session is never mentioned in the /172.18.68.139 log.
> Node 172.18.68.138 (data centre Z):
> The issue here seems to be repair session
> #a55c16e1-35eb-11e4-8e7e-51c077eaf311. It got responses for all its merkle
> tree requests, did some streaming, but seems to have stopped after finishing
> with one table (rollups60). I found it as follows: it is the only repair for
> which there is no "session completed successfully" message in the log.
> Some log file snippets are attached.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)