[jira] [Comment Edited] (CASSANDRA-16143) Streaming fails when s SSTable writer finish() exceeds internode_tcp_user_timeout

2020-12-05 Thread Yifan Cai (Jira)


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

Yifan Cai edited comment on CASSANDRA-16143 at 12/5/20, 10:08 AM:
--

Made a jvm dtest that is able to reproduce the TCP connection timeout during 
streaming reliably. The test code can be found 
[here|https://github.com/apache/cassandra/compare/trunk...yifan-c:C-16143/test].
 The test blocks CassandraCompressedStreamReader.read in the receiver for 10 
seconds to simulate the slow disk, and reduces the send/receive buffer to keep 
the connection active. 

Run the {{CASSANDRA-16143.sh}} and it will start the test within a docker 
container. The container is needed in order to enable EPOLL. (Not necessary if 
you are already running on Linux).

The test should fail with stacktrace like the following

{code:java}
[junit-timeout] ERROR 21:16:18 [Stream #f4c1ac00-2850-11eb-9cb2-ed254fd63997] 
Streaming error occurred on session with peer 127.0.0.2:7012
[junit-timeout] org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: 
The channel this output stream was writing to has been closed
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.flush(AsyncChannelOutputPlus.java:230)
[junit-timeout] at 
org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:179)
[junit-timeout] at 
org.apache.cassandra.streaming.messages.OutgoingStreamMessage.serialize(OutgoingStreamMessage.java:87)
[junit-timeout] at 
org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:45)
[junit-timeout] at 
org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:34)
[junit-timeout] at 
org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:42)
[junit-timeout] at 
org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:347)
[junit-timeout] at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
[junit-timeout] at 
java.util.concurrent.FutureTask.run(FutureTask.java:266)
[junit-timeout] at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[junit-timeout] at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[junit-timeout] at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
[junit-timeout] at java.lang.Thread.run(Thread.java:748)
[junit-timeout] Suppressed: java.nio.channels.ClosedChannelException: 
null
[junit-timeout] at 
org.apache.cassandra.net.AsyncStreamingOutputPlus.doFlush(AsyncStreamingOutputPlus.java:81)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.flush(AsyncChannelOutputPlus.java:229)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.close(AsyncChannelOutputPlus.java:248)
[junit-timeout] at 
org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:348)
[junit-timeout] ... 6 common frames omitted
[junit-timeout] Caused by: io.netty.channel.unix.Errors$NativeIoException: 
writeAddress(..) failed: Connection timed out
{code}


was (Author: yifanc):
Made a jvm dtest that is able to reproduce the TCP connection timeout during 
streaming reliably. The test code can be found 
[here|https://github.com/yifan-c/cassandra/commit/b6dd29908f21f779c8be27243f6548d6661b264f].
 The test blocks CassandraCompressedStreamReader.read in the receiver for 10 
seconds to simulate the slow disk, and reduces the send/receive buffer to keep 
the connection active. 

Run the {{CASSANDRA-16143.sh}} and it will start the test within a docker 
container. The container is needed in order to enable EPOLL. (Not necessary if 
you are already running on Linux).

The test should fail with stacktrace like the following

{code:java}
[junit-timeout] ERROR 21:16:18 [Stream #f4c1ac00-2850-11eb-9cb2-ed254fd63997] 
Streaming error occurred on session with peer 127.0.0.2:7012
[junit-timeout] org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: 
The channel this output stream was writing to has been closed
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputP

[jira] [Comment Edited] (CASSANDRA-16143) Streaming fails when s SSTable writer finish() exceeds internode_tcp_user_timeout

2020-12-01 Thread Adam Holmberg (Jira)


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

Adam Holmberg edited comment on CASSANDRA-16143 at 12/1/20, 10:20 PM:
--

Added a few comments and questions to the PR.
I just noticed the PR is somehow linked and updating the Work Log section of 
this ticket. Is that new?


was (Author: aholmber):
Added a few comments and questions to the PR.

> Streaming fails when s SSTable writer finish() exceeds 
> internode_tcp_user_timeout
> -
>
> Key: CASSANDRA-16143
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16143
> Project: Cassandra
>  Issue Type: Bug
>  Components: Messaging/Internode
>Reporter: Jon Meredith
>Assignee: Yifan Cai
>Priority: Normal
> Fix For: 4.0-beta
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> tl;dr The internode TCP user timeout that provides more responsive detection 
> of dead nodes for internode message will cause streaming to fail if system 
> calls to fsync/fdatasync exceed the timeout (default 30s).
> To workaround, explicitly set internode_tcp_user_timeout to longer than 
> fsync/fdatasync, or to zero to revert to the operating system default.
> Details:
> While bootstrapping a replacement 4.0beta3 node in an existing cluster, 
> bootstrap streaming repeatedly failed with the streaming follower logging
> {code:java}
> ERROR 2020-09-10T14:29:34,711 [NettyStreaming-Outbound-1.1.1.1.7000:1] 
> org.apache.cassandra.streaming.StreamSession:693 - [Stream 
> #7cb67c00-f3ac-11ea-b940-f7836f164528] Streaming error occurred on session 
> with peer 1.1.1.1:7000
> org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: The channel 
> this output stream was writing to has been closed
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200)
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158)
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.waitForSpace(AsyncChannelOutputPlus.java:140)
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.beginFlush(AsyncChannelOutputPlus.java:97)
>at 
> org.apache.cassandra.net.AsyncStreamingOutputPlus.lambda$writeToChannel$0(AsyncStreamingOutputPlus.java:142)
>at 
> org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:90)
>at 
> org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:138)
>at 
> org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:89)
>at 
> org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:180)
>at 
> org.apache.cassandra.streaming.messages.OutgoingStreamMessage.serialize(OutgoingStreamMessage.java:87)
>at 
> org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:45)
>at 
> org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:34)
>at 
> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:40)
>at 
> org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:347)
>at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
>at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  [netty-all-4.1.50.Final.jar:4.1.50.Final]
>at java.lang.Thread.run(Thread.java:834) [?:?]
>Suppressed: java.nio.channels.ClosedChannelException
>at 
> org.apache.cassandra.net.AsyncStreamingOutputPlus.doFlush(AsyncStreamingOutputPlus.java:78)
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.flush(AsyncChannelOutputPlus.java:229)
>at 
> org.apache.cassandra.net.AsyncChannelOutputPlus.close(AsyncChannelOutputPlus.java:248)
>at 
> org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:348)
>at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
>at java.util.concurrent.FutureTask.run(FutureTask.java:264) 
> [?:?