[ https://issues.apache.org/jira/browse/CASSANDRA-14520?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16511666#comment-16511666 ]
Jason Brown commented on CASSANDRA-14520: ----------------------------------------- Here's a stack trace [~bdeggleston] sent me before with this, unedited from the original: {noformat} WARN [Stream-Deserializer-/17.177.98.2:47610-9785e92c] 2018-02-14 16:46:26,470 CompressedStreamReader.java:111 - [Stream 757ae2c0-11e9-11e8-8fc8-f7aed080d7f2] Error while reading partition DecoratedKey(-7999952965546089109, 2f645c5d1654236063405e7f6c4f273637231a5f037e696a6f3b4e411a18462b0942754651100f381d24757311071a761f46170a46524d62083458313e0442245f6b4046401f46712b1e310456326c245e080d02294d216266252d1635662e093a7f2b027d00451f2c350c1a5e177e574a785d1b702b1b400105281e4c3f6359704875124165705e2c403d0346725454531f076f3d577203233a73213027113520172a7e01467a5c776d4d546d5f57343f64215f4c5b05502922694a5d38621440362d0960005b7d1663565c79501e5c3c7a5032603e6f5e7b19645a2c017b391547212e6a7b516270317a42) from stream on ks='fullks' and table='kvtest'. ERROR [Stream-Deserializer-/17.177.98.2:47610-9785e92c] 2018-02-14 16:46:26,476 StreamSession.java:617 - [Stream #757ae2c0-11e9-11e8-8fc8-f7aed080d7f2] Streaming error occurred on session with peer 17.177.98.2 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:748) [na:1.8.0_131] 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:94) ~[main/:na] at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:58) ~[main/:na] ... 4 common frames omitted INFO [Stream-Deserializer-/17.177.98.2:47610-9785e92c] 2018-02-14 16:46:26,478 StreamResultFuture.java:193 - [Stream #757ae2c0-11e9-11e8-8fc8-f7aed080d7f2] Session with /17.177.98.2 is complete WARN [Stream-Deserializer-/17.177.98.2:47610-9785e92c] 2018-02-14 16:46:26,479 StreamResultFuture.java:220 - [Stream #757ae2c0-11e9-11e8-8fc8-f7aed080d7f2] Stream failed ERROR [NettyStreaming-Outbound-/17.177.98.2:1] 2018-02-14 16:46:26,486 CassandraDaemon.java:211 - Exception in thread Thread[NettyStreaming-Outbound-/17.177.98.2: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_131] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_131] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_131] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_131] at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na] at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131] Caused by: java.nio.channels.ClosedByInterruptException: null at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) ~[na:1.8.0_131] at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:746) ~[na:1.8.0_131] at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727) ~[na:1.8.0_131] at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:129) ~[main/:na] ... 12 common frames omitted ERROR [NettyStreaming-Outbound-/17.177.98.2:1] 2018-02-14 16:46:26,488 StorageService.java:393 - Stopping gossiper WARN [NettyStreaming-Outbound-/17.177.98.2:1] 2018-02-14 16:46:26,489 StorageService.java:315 - Stopping gossip by operator request INFO [NettyStreaming-Outbound-/17.177.98.2:1] 2018-02-14 16:46:26,489 Gossiper.java:1514 - Announcing shutdown {noformat} Note: the {{"stream can only read forward"}} error will be resolved by CASSANDRA-13938, which is PATCH_AVAILABLE as of the time I type this. > ClosedChannelException handled as FSError > ----------------------------------------- > > Key: CASSANDRA-14520 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14520 > Project: Cassandra > Issue Type: Bug > Reporter: Blake Eggleston > Assignee: Jason Brown > Priority: Major > Fix For: 4.0 > > > After the messaging service netty refactor, I’ve seen a few instances where a > closed socket causes a ClosedChannelException (an IOException subclass) to be > thrown. The exception is caught by ChannelProxy, interpreted as a disk error, > and is then re-thrown as an FSError, causing the node to be shutdown. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org