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

Stefano Ortolalani commented on CASSANDRA-8696:
-----------------------------------------------

It seems it hit me as well. 
Here too after changing the replication factor.
As a result the logs show that exception plus a repair job is still going on 
(looping I presume) for the last 12 hours.
I enabled the additional logs. Below an excerpt:

INFO  [RepairJobTask:2] 2015-04-01 20:10:38,515 StreamingRepairTask.java:68 - 
[streaming task #0063b860-d8ab-11e4-9931-47110801b395] Performing streaming 
repair of 5 ranges with /XX.YY.81.6
INFO  [RepairJobTask:1] 2015-04-01 20:10:38,515 ColumnFamilyStore.java:877 - 
Enqueuing flush of rollups300: 474175 (0%) on-heap, 0 (0%) off-heap
INFO  [MemtableFlushWriter:474] 2015-04-01 20:10:38,516 Memtable.java:339 - 
Writing Memtable-rollups300@1473895455(40384 serialized bytes, 1262 ops, 0%/0% 
of on/off-heap limit)
INFO  [MemtableFlushWriter:474] 2015-04-01 20:10:38,788 Memtable.java:378 - 
Completed flushing 
/data/cassandra/data/OpsCenter/rollups300-733a31d374af11e48baa6776bf484396/OpsCenter-rollups300-ka-63048-Data.db
 (41977 bytes) for commitlog position ReplayPosition(segmentId=1427807498701, 
position=22965644)
INFO  [AntiEntropyStage:1] 2015-04-01 20:10:39,718 RepairSession.java:237 - 
[repair #0063b860-d8ab-11e4-9931-47110801b395] rollups60 is fully synced (1 
remaining column family to sync for this session)
INFO  [AntiEntropyStage:1] 2015-04-01 20:10:42,741 RepairSession.java:171 - 
[repair #0063b860-d8ab-11e4-9931-47110801b395] Received merkle tree for 
rollups7200 from /XX.YY.81.3
INFO  [AntiEntropyStage:1] 2015-04-01 20:10:42,765 RepairSession.java:171 - 
[repair #0063b860-d8ab-11e4-9931-47110801b395] Received merkle tree for 
rollups7200 from /XX.YY.81.6
INFO  [AntiEntropyStage:1] 2015-04-01 20:10:42,767 RepairSession.java:171 - 
[repair #0063b860-d8ab-11e4-9931-47110801b395] Received merkle tree for 
rollups7200 from /XX.YY.33.6
INFO  [RepairJobTask:6] 2015-04-01 20:10:42,767 Differencer.java:67 - [repair 
#0063b860-d8ab-11e4-9931-47110801b395] Endpoints /XX.YY.81.3 and /XX.YY.81.6 
are consistent for rollups7200
INFO  [RepairJobTask:5] 2015-04-01 20:10:42,767 Differencer.java:74 - [repair 
#0063b860-d8ab-11e4-9931-47110801b395] Endpoints /XX.YY.81.3 and /XX.YY.33.6 
have 1 range(s) out of sync for rollups7200
INFO  [RepairJobTask:4] 2015-04-01 20:10:42,767 Differencer.java:74 - [repair 
#0063b860-d8ab-11e4-9931-47110801b395] Endpoints /XX.YY.81.6 and /XX.YY.33.6 
have 1 range(s) out of sync for rollups7200
INFO  [RepairJobTask:5] 2015-04-01 20:10:42,769 StreamingRepairTask.java:68 - 
[streaming task #0063b860-d8ab-11e4-9931-47110801b395] Performing streaming 
repair of 1 ranges with /XX.YY.81.3
INFO  [RepairJobTask:4] 2015-04-01 20:10:42,769 StreamingRepairTask.java:68 - 
[streaming task #0063b860-d8ab-11e4-9931-47110801b395] Performing streaming 
repair of 1 ranges with /XX.YY.81.6
ERROR [RepairJobTask:6] 2015-04-01 20:10:53,557 RepairJob.java:145 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /XX.YY.81.6
        at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:349) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
ERROR [AntiEntropySessions:353] 2015-04-01 20:10:53,558 RepairSession.java:303 
- [repair #0063b860-d8ab-11e4-9931-47110801b395] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
INFO  [AntiEntropySessions:354] 2015-04-01 20:10:53,558 RepairSession.java:260 
- [repair #330ec250-d8ab-11e4-9931-47110801b395] new session: will sync 
/XX.YY.33.6, /XX.YY.81.3, /XX.YY.81.7 on range 
(5998308689526371329,6004774069065445793] for OpsCenter.[events, rollups60, 
rollups300, rollups7200, events_timeline, bestpractice_results, pdps, settings, 
backup_reports, rollups86400]
ERROR [AntiEntropySessions:353] 2015-04-01 20:10:53,558 
CassandraDaemon.java:167 - Exception in thread 
Thread[AntiEntropySessions:353,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
creation.
        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.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
Caused by: java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        ... 3 common frames omitted
ERROR [RepairJobTask:3] 2015-04-01 20:11:04,558 RepairJob.java:145 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /XX.YY.81.7
        at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:349) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
INFO  [AntiEntropySessions:355] 2015-04-01 20:11:04,559 RepairSession.java:260 
- [repair #399d60e0-d8ab-11e4-9931-47110801b395] new session: will sync 
/XX.YY.33.6, /XX.YY.81.3, /XX.YY.81.6 on range 
(-5313304202554791250,-5303868000298767019] for OpsCenter.[events, rollups60, 
rollups300, rollups7200, events_timeline, bestpractice_results, pdps, settings, 
backup_reports, rollups86400]
ERROR [AntiEntropySessions:354] 2015-04-01 20:11:04,559 RepairSession.java:303 
- [repair #330ec250-d8ab-11e4-9931-47110801b395] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
ERROR [AntiEntropySessions:354] 2015-04-01 20:11:04,560 
CassandraDaemon.java:167 - Exception in thread 
Thread[AntiEntropySessions:354,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
creation.
        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.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
Caused by: java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        ... 3 common frames omitted
ERROR [RepairJobTask:3] 2015-04-01 20:11:15,559 RepairJob.java:145 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /XX.YY.33.6
        at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:349) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
INFO  [AntiEntropySessions:349] 2015-04-01 20:11:15,560 RepairSession.java:260 
- [repair #402c2680-d8ab-11e4-9931-47110801b395] new session: will sync 
/XX.YY.33.6, /XX.YY.33.7, /XX.YY.81.5 on range 
(-6853139877646932913,-6843031538522862533] for OpsCenter.[events, rollups60, 
rollups300, rollups7200, events_timeline, bestpractice_results, pdps, settings, 
backup_reports, rollups86400]
ERROR [AntiEntropySessions:355] 2015-04-01 20:11:15,560 RepairSession.java:303 
- [repair #399d60e0-d8ab-11e4-9931-47110801b395] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
ERROR [AntiEntropySessions:355] 2015-04-01 20:11:15,561 
CassandraDaemon.java:167 - Exception in thread 
Thread[AntiEntropySessions:355,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
creation.
        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.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
Caused by: java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        ... 3 common frames omitted
ERROR [RepairJobTask:3] 2015-04-01 20:11:26,560 RepairJob.java:145 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /XX.YY.33.6
        at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:349) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
INFO  [AntiEntropySessions:353] 2015-04-01 20:11:26,563 RepairSession.java:260 
- [repair #46bac510-d8ab-11e4-9931-47110801b395] new session: will sync 
/XX.YY.33.6, /XX.YY.33.4, /XX.YY.81.2 on range 
(8624472375972870675,8630654047035545510] for OpsCenter.[events, rollups60, 
rollups300, rollups7200, events_timeline, bestpractice_results, pdps, settings, 
backup_reports, rollups86400]
ERROR [AntiEntropySessions:349] 2015-04-01 20:11:26,563 RepairSession.java:303 
- [repair #402c2680-d8ab-11e4-9931-47110801b395] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
ERROR [AntiEntropySessions:349] 2015-04-01 20:11:26,563 
CassandraDaemon.java:167 - Exception in thread 
Thread[AntiEntropySessions:349,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
creation.
        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.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_67]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_67]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_67]
        at java.lang.Thread.run(Thread.java:745) [na:1.7.0_67]
Caused by: java.io.IOException: Failed during snapshot creation.
        at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
        ... 3 common frames omitted
DEBUG [AntiEntropyStage:1] 2015-04-01 20:11:26,707 
RepairMessageVerbHandler.java:92 - Enqueuing response to snapshot request 
0063b860-d8ab-11e4-9931-47110801b395 to /XX.YY.33.6
DEBUG [AntiEntropyStage:1] 2015-04-01 20:11:26,709 
RepairMessageVerbHandler.java:92 - Enqueuing response to snapshot request 
330ec250-d8ab-11e4-9931-47110801b395 to /XX.YY.33.6
DEBUG [AntiEntropyStage:1] 2015-04-01 20:11:26,710 
RepairMessageVerbHandler.java:92 - Enqueuing response to snapshot request 
399d60e0-d8ab-11e4-9931-47110801b395 to /XX.YY.33.6
DEBUG [AntiEntropyStage:1] 2015-04-01 20:11:26,711 
RepairMessageVerbHandler.java:92 - Enqueuing response to snapshot request 
402c2680-d8ab-11e4-9931-47110801b395 to /XX.YY.33.6
DEBUG [AntiEntropyStage:1] 2015-04-01 20:11:26,712 
RepairMessageVerbHandler.java:92 - Enqueuing response to snapshot request 
46bac510-d8ab-11e4-9931-47110801b395 to /XX.YY.33.6
INFO  [RepairJobTask:3] 2015-04-01 20:11:26,712 RepairJob.java:163 - [repair 
#46bac510-d8ab-11e4-9931-47110801b395] requesting merkle trees for events (to 
[/XX.YY.33.4, /XX.YY.81.2, /XX.YY.33.6])
INFO  [AntiEntropyStage:1] 2015-04-01 20:11:26,715 RepairSession.java:171 - 
[repair #46bac510-d8ab-11e4-9931-47110801b395] Received merkle tree for events 
from /XX.YY.33.4
INFO  [AntiEntropyStage:1] 2015-04-01 20:11:26,718 RepairSession.java:171 - 
[repair #46bac510-d8ab-11e4-9931-47110801b395] Received merkle tree for events 
from /XX.YY.81.2
INFO  [AntiEntropyStage:1] 2015-04-01 20:11:26,719 RepairSession.java:171 - 
[repair #46bac510-d8ab-11e4-9931-47110801b395] Received merkle tree for events 
from /XX.YY.33.6


> nodetool repair on cassandra 2.1.2 keyspaces return 
> java.lang.RuntimeException: Could not create snapshot
> ---------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-8696
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8696
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Jeff Liu
>             Fix For: 2.1.5
>
>
> When trying to run nodetool repair -pr on cassandra node ( 2.1.2), cassandra 
> throw java exceptions: cannot create snapshot. 
> the error log from system.log:
> {noformat}
> INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:28,815 
> StreamResultFuture.java:166 - [Stream #692c1450-a692-11e4-9973-070e938df227 
> ID#0] Prepare completed. Receiving 2 files(221187 bytes), sending 5 
> files(632105 bytes)
> INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,046 
> StreamResultFuture.java:180 - [Stream #692c1450-a692-11e4-9973-070e938df227] 
> Session with /10.97.9.110 is complete
> INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,046 
> StreamResultFuture.java:212 - [Stream #692c1450-a692-11e4-9973-070e938df227] 
> All sessions completed
> INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,047 
> StreamingRepairTask.java:96 - [repair #685e3d00-a692-11e4-9973-070e938df227] 
> streaming task succeed, returning response to /10.98.194.68
> INFO  [RepairJobTask:1] 2015-01-28 02:07:29,065 StreamResultFuture.java:86 - 
> [Stream #692c6270-a692-11e4-9973-070e938df227] Executing streaming plan for 
> Repair
> INFO  [StreamConnectionEstablisher:4] 2015-01-28 02:07:29,065 
> StreamSession.java:213 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
> Starting streaming to /10.66.187.201
> INFO  [StreamConnectionEstablisher:4] 2015-01-28 02:07:29,070 
> StreamCoordinator.java:209 - [Stream #692c6270-a692-11e4-9973-070e938df227, 
> ID#0] Beginning stream session with /10.66.187.201
> INFO  [STREAM-IN-/10.66.187.201] 2015-01-28 02:07:29,465 
> StreamResultFuture.java:166 - [Stream #692c6270-a692-11e4-9973-070e938df227 
> ID#0] Prepare completed. Receiving 5 files(627994 bytes), sending 5 
> files(632105 bytes)
> INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,971 
> StreamResultFuture.java:180 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
> Session with /10.66.187.201 is complete
> INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,972 
> StreamResultFuture.java:212 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
> All sessions completed
> INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,972 
> StreamingRepairTask.java:96 - [repair #685e3d00-a692-11e4-9973-070e938df227] 
> streaming task succeed, returning response to /10.98.194.68
> ERROR [RepairJobTask:1] 2015-01-28 02:07:39,444 RepairJob.java:127 - Error 
> occurred during snapshot phase
> java.lang.RuntimeException: Could not create snapshot at /10.97.9.110
>         at 
> org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
>  ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at 
> org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:347) 
> ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> ~[na:1.7.0_45]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> ~[na:1.7.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  [na:1.7.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> INFO  [AntiEntropySessions:6] 2015-01-28 02:07:39,445 RepairSession.java:260 
> - [repair #6f85e740-a692-11e4-9973-070e938df227] new session: will sync 
> /10.98.194.68, /10.66.187.201, /10.226.218.135 on range 
> (12817179804668051873746972069086
> 2638799,128635403083592540777731520865977436165] for events.[bigint0text, 
> bigint0boolean, bigint0int, dataset_catalog, column_categories, 
> bigint0double, bigint0bigint]
> ERROR [AntiEntropySessions:5] 2015-01-28 02:07:39,445 RepairSession.java:303 
> - [repair #685e3d00-a692-11e4-9973-070e938df227] session completed with the 
> following error
> java.io.IOException: Failed during snapshot creation.
>         at 
> org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
>  ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at 
> org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) 
> ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
> ~[guava-16.0.jar:na]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  [na:1.7.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> ERROR [AntiEntropySessions:5] 2015-01-28 02:07:39,446 
> CassandraDaemon.java:153 - Exception in thread 
> Thread[AntiEntropySessions:5,5,RMI Runtime]
> java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
> creation.
>         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.2.jar:2.1.2]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> ~[na:1.7.0_45]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> ~[na:1.7.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  ~[na:1.7.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_45]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> Caused by: java.io.IOException: Failed during snapshot creation.
>         at 
> org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
>  ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at 
> org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) 
> ~[apache-cassandra-2.1.2.jar:2.1.2]
>         at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
> ~[guava-16.0.jar:na]
>         ... 3 common frames omitted
> {noformat}
> The only change we did recently was to change tablespace replication factor 
> from 2 to 3 before seeing those errors. Also same time we start seeing 
> timeout errors from application. 
> the timeout error is something like:
> {noformat}
> core.exceptions.ReadTimeoutException: Cassandra timeout during read query at 
> consistency ONE (1 responses were required but only 0 replica responded)
>     at 
> com.datastax.driver.core.exceptions.ReadTimeoutException.copy(ReadTimeoutException.java:69)
>  ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> com.datastax.driver.core.Responses$Error.asException(Responses.java:100) 
> ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:110)
>  ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:249)
>  ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at com.datastax.driver.core.RequestHandler.onSet(RequestHandler.java:433) 
> ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> com.datastax.driver.core.Connection$Dispatcher.messageReceived(Connection.java:668)
>  ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:70)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:462)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:443)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:318)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) 
> ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  ~[na:1.7.0_55]
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  ~[na:1.7.0_55]
>     at java.lang.Thread.run(Thread.java:745) ~[na:1.7.0_55]
> Caused by: com.datastax.driver.core.exceptions.ReadTimeoutException: 
> Cassandra timeout during read query at consistency ONE (1 responses were 
> required but only 0 replica responded)
>     at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:61) 
> ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:38) 
> ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:168) 
> ~[com.datastax.cassandra.cassandra-driver-core-2.1.3.jar:na]
>     at 
> org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66)
>  ~[io.netty.netty-3.9.0.Final.jar:na]
>     ... 21 common frames omitted
> {noformat}



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

Reply via email to