Details
-
Bug
-
Status: Resolved
-
Urgent
-
Resolution: Fixed
-
None
-
Availability - Process Crash
-
Critical
Description
Running through a simple scenario to test some of the new repair features, I was not able to make a repair command work. Further, the exception seemed to trigger a nasty failure state that basically shuts down the netty connections for messaging and CQL on the nodes transferring back data to the node being repaired. The following steps reproduce this issue consistently.
Cassandra stress profile (probably not necessary, but this one provides a really simple schema and consistent data shape):
keyspace: standard_long keyspace_definition: | CREATE KEYSPACE standard_long WITH replication = {'class':'SimpleStrategy', 'replication_factor':3}; table: test_data table_definition: | CREATE TABLE test_data ( key text, ts bigint, val text, PRIMARY KEY (key, ts) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY (ts DESC) AND bloom_filter_fp_chance=0.010000 AND caching={'keys':'ALL', 'rows_per_partition':'NONE'} AND comment='' AND dclocal_read_repair_chance=0.000000 AND gc_grace_seconds=864000 AND read_repair_chance=0.000000 AND compaction={'class': 'SizeTieredCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; columnspec: - name: key population: uniform(1..50000000) # 50 million records available - name: ts cluster: gaussian(1..50) # Up to 50 inserts per record - name: val population: gaussian(128..1024) # varrying size of value data insert: partitions: fixed(1) # only one insert per batch for individual partitions select: fixed(1)/1 # each insert comes in one at a time batchtype: UNLOGGED queries: single: cql: select * from test_data where key = ? and ts = ? limit 1; series: cql: select key,ts,val from test_data where key = ? limit 10;
The commands to build and run:
ccm create 4_0_test -v git:trunk -n 3 -s ccm stress user profile=./histo-test-schema.yml ops\(insert=20,single=1,series=1\) duration=15s -rate threads=4 # flush the memtable just to get everything on disk ccm node1 nodetool flush ccm node2 nodetool flush ccm node3 nodetool flush # disable hints for nodes 2 and 3 ccm node2 nodetool disablehandoff ccm node3 nodetool disablehandoff # stop node1 ccm node1 stop ccm stress user profile=./histo-test-schema.yml ops\(insert=20,single=1,series=1\) duration=45s -rate threads=4 # wait 10 seconds ccm node1 start # Note that we are local to ccm's nodetool install 'cause repair preview is not reported yet node1/bin/nodetool repair --preview node1/bin/nodetool repair standard_long test_data
The error outputs from the last repair command follow. First, this is stdout from node1:
$ node1/bin/nodetool repair standard_long test_data objc[47876]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_101.jdk/Contents/Home/bin/java (0x10274d4c0) and /Library/Java/JavaVirtualMachines/jdk1.8.0_101.jdk/Contents/Home/jre/lib/libinstrument.dylib (0x1047b64e0). One of the two will be used. Which one is undefined. [2017-10-05 14:31:52,425] Starting repair command #4 (7e1a9150-a98e-11e7-ad86-cbd2801b8de2), repairing keyspace standard_long with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [test_data], dataCenters: [], hosts: [], previewKind: NONE, # of ranges: 3, pull repair: false, force repair: false) [2017-10-05 14:32:07,045] Repair session 7e2e8e80-a98e-11e7-ad86-cbd2801b8de2 for range [(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]] failed with error Stream failed [2017-10-05 14:32:07,048] null [2017-10-05 14:32:07,050] Repair command #4 finished in 14 seconds error: Repair job has failed with the error message: [2017-10-05 14:32:07,048] null -- StackTrace -- java.lang.RuntimeException: Repair job has failed with the error message: [2017-10-05 14:32:07,048] null at org.apache.cassandra.tools.RepairRunner.progress(RepairRunner.java:122) at org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener.handleNotification(JMXNotificationProgressListener.java:77) at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.dispatchNotification(ClientNotifForwarder.java:583) at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.doRun(ClientNotifForwarder.java:533) at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.run(ClientNotifForwarder.java:452) at com.sun.jmx.remote.internal.ClientNotifForwarder$LinearExecutor$1.run(ClientNotifForwarder.java:108)
node1's system.log:
INFO [Stream-Deserializer-/127.0.0.2:63069-e0af297f] 2017-10-05 14:32:07,037 StreamResultFuture.java:193 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.2 is complete INFO [Stream-Deserializer-/127.0.0.3:63068-eb8f23bc] 2017-10-05 14:32:07,037 StreamResultFuture.java:193 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.3 is complete ERROR [Streaming-Netty-Thread-5-5] 2017-10-05 14:32:07,037 StreamSession.java:617 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.3 java.nio.channels.ClosedChannelException: null at io.netty.channel.AbstractChannel$AbstractUnsafe.write(...)(Unknown Source) ~[netty-all-4.1.14.Final.jar:4.1.14.Final] ERROR [Streaming-Netty-Thread-5-7] 2017-10-05 14:32:07,038 StreamSession.java:617 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.2 java.nio.channels.ClosedChannelException: null at io.netty.channel.AbstractChannel$AbstractUnsafe.write(...)(Unknown Source) ~[netty-all-4.1.14.Final.jar:4.1.14.Final] WARN [Stream-Deserializer-/127.0.0.2:63069-e0af297f] 2017-10-05 14:32:07,038 StreamResultFuture.java:220 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Stream failed WARN [Stream-Deserializer-/127.0.0.3:63068-eb8f23bc] 2017-10-05 14:32:07,038 StreamResultFuture.java:220 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Stream failed WARN [RepairJobTask:1] 2017-10-05 14:32:07,038 RepairJob.java:176 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] test_data sync failed ERROR [Stream-Deserializer-/127.0.0.3:7000-48246b87] 2017-10-05 14:32:07,041 StreamSession.java:757 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Remote peer 127.0.0.3 failed stream session. ERROR [RepairJobTask:1] 2017-10-05 14:32:07,042 RepairSession.java:326 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Session completed with the following error org.apache.cassandra.streaming.StreamException: Stream failed at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88) ~[main/:na] at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) ~[guava-18.0.jar:na] at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:221) ~[main/:na] at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:197) ~[main/:na] at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:488) ~[main/:na] at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:601) ~[main/:na] at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:207) ~[main/:na] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101] ERROR [RepairJobTask:1] 2017-10-05 14:32:07,043 RepairRunnable.java:564 - Repair session 7e2e8e80-a98e-11e7-ad86-cbd2801b8de2 for range [(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]] failed with error Stream failed org.apache.cassandra.streaming.StreamException: Stream failed at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88) ~[main/:na] at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) ~[guava-18.0.jar:na] at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:221) ~[main/:na] at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:197) ~[main/:na] at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:488) ~[main/:na] at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:601) ~[main/:na] at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:207) ~[main/:na] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101] INFO [RepairJobTask:1] 2017-10-05 14:32:07,045 CoordinatorSession.java:233 - Incremental repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 failed ERROR [Stream-Deserializer-/127.0.0.2:7000-4b83e3cb] 2017-10-05 14:32:07,045 StreamSession.java:757 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Remote peer 127.0.0.2 failed stream session. INFO [AntiEntropyStage:1] 2017-10-05 14:32:07,048 CoordinatorSession.java:233 - Incremental repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 failed INFO [AntiEntropyStage:1] 2017-10-05 14:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 INFO [RepairJobTask:1] 2017-10-05 14:32:07,049 RepairRunnable.java:647 - Repair command #4 finished in 14 seconds
node2's system.log (note the transport shutdowns at the end):
INFO [AntiEntropyStage:1] 2017-10-05 18:31:52,521 LocalSessions.java:560 - Beginning local incremental repair session LocalSession{sessionID=7e1a9150-a98e-11e7-ad86-cbd2801b8de2, state=PREPARING, coordinator=/127.0.0.1, tableIds=[99d53860-a98d-11e7-9807-39cb3e573e5c], repairedAt=1507181512483, ranges=[(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]], participants=[/127.0.0.1, /127.0.0.2, /127.0.0.3], startedAt=1507181512, lastUpdate=1507181512} INFO [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:642 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Starting anticompaction for standard_long.test_data on 2/2 sstables INFO [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node2/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-27-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting INFO [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node2/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-26-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting INFO [CompactionExecutor:224] 2017-10-05 18:31:52,547 CompactionManager.java:699 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Completed anticompaction successfully INFO [AntiEntropyStage:1] 2017-10-05 18:31:57,500 Validator.java:292 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Sending completed merkle tree to /127.0.0.1 for standard_long.test_data INFO [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:05,417 StreamResultFuture.java:115 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Creating new streaming plan for Repair INFO [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:05,418 StreamResultFuture.java:122 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2, ID#0] Received streaming plan for Repair INFO [NonPeriodicTasks:1] 2017-10-05 18:32:05,856 StreamResultFuture.java:179 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.689MiB) INFO [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:06,625 StreamResultFuture.java:179 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.689MiB) WARN [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,747 CompressedStreamReader.java:112 - [Stream 85d4b790-a98e-11e7-ad86-cbd2801b8de2] Error while reading partition DecoratedKey(-9060243433852736644, 5f1c6c5d747c) from stream on ks='standard_long' and table='test_data'. ERROR [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,759 StreamSession.java:617 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.1 org.apache.cassandra.streaming.StreamReceiveException: java.lang.AssertionError: stream can only read forward. at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:63) ~[main/:na] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:41) ~[main/:na] at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) ~[main/:na] at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:178) ~[main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101] Caused by: java.lang.AssertionError: stream can only read forward. at org.apache.cassandra.streaming.compress.CompressedInputStream.position(CompressedInputStream.java:108) ~[main/:na] at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:96) ~[main/:na] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:58) ~[main/:na] ... 4 common frames omitted INFO [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,761 StreamResultFuture.java:193 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.1 is complete WARN [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,762 StreamResultFuture.java:220 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Stream failed ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,765 CassandraDaemon.java:211 - Exception in thread Thread[NettyStreaming-Outbound-/127.0.0.1:1,5,main] org.apache.cassandra.io.FSReadError: java.nio.channels.ClosedByInterruptException at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:133) ~[main/:na] at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:94) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:111) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:53) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[main/:na] at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:41) ~[main/:na] at org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:324) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_101] at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101] Caused by: java.nio.channels.ClosedByInterruptException: null at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) ~[na:1.8.0_101] at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:746) ~[na:1.8.0_101] at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727) ~[na:1.8.0_101] at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:129) ~[main/:na] ... 12 common frames omitted ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 StorageService.java:393 - Stopping gossiper WARN [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 StorageService.java:315 - Stopping gossip by operator request INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 Gossiper.java:1527 - Announcing shutdown INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,770 StorageService.java:2202 - Node /127.0.0.2 state jump to shutdown INFO [AntiEntropyStage:1] 2017-10-05 18:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,771 StorageService.java:398 - Stopping native transport INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,774 Server.java:180 - Stop listening for CQL clients
And node3 system.log (similar to node2):
INFO [AntiEntropyStage:1] 2017-10-05 18:31:52,521 LocalSessions.java:560 - Beginning local incremental repair session LocalSession{sessionID=7e1a9150-a98e-11e7-ad86-cbd2801b8de2, state=PREPARING, coordinator=/127.0.0.1, tableIds=[99d53860-a98d-11e7-9807-39cb3e573e5c], repairedAt=1507181512483, ranges=[(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]], participants=[/127.0.0.1, /127.0.0.2, /127.0.0.3], startedAt=1507181512, lastUpdate=1507181512} INFO [CompactionExecutor:249] 2017-10-05 18:31:52,542 CompactionManager.java:642 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Starting anticompaction for standard_long.test_data on 2/2 sstables INFO [CompactionExecutor:249] 2017-10-05 18:31:52,543 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node3/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-27-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting INFO [CompactionExecutor:249] 2017-10-05 18:31:52,543 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node3/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-26-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting INFO [CompactionExecutor:249] 2017-10-05 18:31:52,550 CompactionManager.java:699 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Completed anticompaction successfully INFO [AntiEntropyStage:1] 2017-10-05 18:31:57,918 Validator.java:292 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Sending completed merkle tree to /127.0.0.1 for standard_long.test_data INFO [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:05,817 StreamResultFuture.java:115 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Creating new streaming plan for Repair INFO [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:05,818 StreamResultFuture.java:122 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2, ID#0] Received streaming plan for Repair INFO [NonPeriodicTasks:1] 2017-10-05 18:32:05,866 StreamResultFuture.java:179 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.679MiB) INFO [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:06,622 StreamResultFuture.java:179 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.679MiB) WARN [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,759 CompressedStreamReader.java:112 - [Stream 85d3f440-a98e-11e7-ad86-cbd2801b8de2] Error while reading partition DecoratedKey(-9060243433852736644, 5f1c6c5d747c) from stream on ks='standard_long' and table='test_data'. ERROR [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,773 StreamSession.java:617 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.1 org.apache.cassandra.streaming.StreamReceiveException: java.lang.AssertionError: stream can only read forward. at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:63) ~[main/:na] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:41) ~[main/:na] at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) ~[main/:na] at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:178) ~[main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101] Caused by: java.lang.AssertionError: stream can only read forward. at org.apache.cassandra.streaming.compress.CompressedInputStream.position(CompressedInputStream.java:108) ~[main/:na] at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:96) ~[main/:na] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:58) ~[main/:na] ... 4 common frames omitted INFO [GossipStage:1] 2017-10-05 18:32:06,774 Gossiper.java:1040 - InetAddress /127.0.0.2 is now DOWN INFO [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,775 StreamResultFuture.java:193 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.1 is complete WARN [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,775 StreamResultFuture.java:220 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Stream failed ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,778 CassandraDaemon.java:211 - Exception in thread Thread[NettyStreaming-Outbound-/127.0.0.1:1,5,main] org.apache.cassandra.io.FSReadError: java.nio.channels.ClosedByInterruptException at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:133) ~[main/:na] at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:94) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:111) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:53) ~[main/:na] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[main/:na] at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:41) ~[main/:na] at org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:324) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_101] at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101] Caused by: java.nio.channels.ClosedByInterruptException: null at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) ~[na:1.8.0_101] at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:746) ~[na:1.8.0_101] at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727) ~[na:1.8.0_101] at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:129) ~[main/:na] ... 12 common frames omitted ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 StorageService.java:393 - Stopping gossiper WARN [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 StorageService.java:315 - Stopping gossip by operator request INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 Gossiper.java:1527 - Announcing shutdown INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,782 StorageService.java:2202 - Node /127.0.0.3 state jump to shutdown INFO [AntiEntropyStage:1] 2017-10-05 18:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,782 StorageService.java:398 - Stopping native transport INFO [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,785 Server.java:180 - Stop listening for CQL clients
The final state of the cluster after running this repair command:
$ ccm node1 nodetool status Datacenter: datacenter1 ======================= Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns (effective) Host ID Rack UN 127.0.0.1 8.62 MiB 1 100.0% ffe7466b-2937-4322-a388-cca1819f6513 rack1 DN 127.0.0.2 44.54 MiB 1 100.0% e374f662-1da5-477d-b1fb-173b8311c4a9 rack1 DN 127.0.0.3 44.53 MiB 1 100.0% d8d99bd6-4b9f-4510-a4c3-62951be1b4d2 rack1
Attachments
Attachments
Issue Links
- is duplicated by
-
CASSANDRA-14394 Streaming fails with AssertionError
- Resolved
-
CASSANDRA-15212 CassandraInputStream Bugs
- Resolved
- mentioned in
-
Page Loading...