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

2020-12-09 Thread Benjamin Lerer (Jira)


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

Benjamin Lerer commented on CASSANDRA-16143:


+1 Thanks a lot for the patch

> 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: 3h 40m
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 
> 

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

2020-12-07 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-16143:
---

+1

> 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: 3h 40m
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 
> 

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

2020-12-07 Thread Yifan Cai (Jira)


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

Yifan Cai commented on CASSANDRA-16143:
---

Thanks David! 
Pushed a new commit. 

bq. should we use NoSpamLogger vs normal logger?

Using {{NoSpamLogger}} sounds better. The warning message emits when disk goes 
slow. In that case, we can expect a lot of those warnings. Logging every 
warning does not provide more help, from the operators' perspective. I have 
changed the logging interval to 1 minute. 

> 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: 3h 40m
>  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) [?:?]
>

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

2020-12-07 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-16143:
---

Finished first round of review, overall LGTM; only minor comments.

1) typo in docs; have an extra 0
2) should we use NoSpamLogger vs normal logger?

> 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: 3.5h
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> 

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

2020-12-06 Thread Yifan Cai (Jira)


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

Yifan Cai commented on CASSANDRA-16143:
---

Thanks all for the feedback! 
New commits were just pushed to address the comments. The patch is ready to be 
re-reviewed. 

> 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: 2h 10m
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> 

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

2020-12-05 Thread Jon Meredith (Jira)


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

Jon Meredith commented on CASSANDRA-16143:
--

Thanks for raising the concern and testing. I'm ok with it taking longer, as 
long as it times out eventually. Even if it's 10 minutes it's likely better 
than default TCP settings.

> 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: 2h
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  

[jira] [Commented] (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=17244454#comment-17244454
 ] 

Yifan Cai commented on CASSANDRA-16143:
---

Thanks for having the discussion about the timeout default. 5 minutes also 
sounds good to me.

I would like to bring up one concern that the netty/streaming behaves oddly 
when tcp_user_timeout is large. 
The docker test (posted above) was updated with _tcp_user_timeout == 300 
seconds and artificial delay == 290 seconds_. 
The test should pass with the settings, but it failed with the same error, 
"writeAddress(..) failed: Connection timed out". 
The log indicates that after the receiver (node2) completed blocking, the 
sender did not continue writing into channel. The last flush took 302096082 
micros == 302 seconds > tcp_user_timeout, so it failed expectedly. 

Although failed, the test proves that the tcp_user_timeout closes the 
connection at the right moment. 

{code:java}
...
[junit-timeout] node2 <--- block CassandraCompressedStreamReader.read for 290 
seconds
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-10b5986c] node2 
2020-12-05 09:45:02,468 CassandraCompressedStreamReader.java:73 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb] Start receiving file #0 from 
/127.0.0.1:7012, repairedAt = 0, size = 5421, ks = 'distributed_test_keyspace', 
pendingRepair = 'null', table = 'cf'.
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-10b5986c] node2 
2020-12-05 09:45:02,573 StreamingInboundHandler.java:187 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: 10b5986c] Received 
IncomingStreamMessage{header=Header (tableId: 
db5364c0-36dd-11eb-ab7f-6fd7ea8cef30, #0, repairedAt: 0, pendingRepair: null, 
sendByFollower: true), 
stream=CassandraIncomingFile{sstable=distributed_test_keyspace/cf}}
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-10b5986c] node2 
2020-12-05 09:45:02,575 NettyStreamingMessageSender.java:258 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: acbcbd5e] Sending Received 
(db5364c0-36dd-11eb-ab7f-6fd7ea8cef30, #0)
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-10b5986c] node2 
2020-12-05 09:45:02,577 StreamReceiveTask.java:88 - received 1 of 5 total 
files, 5421 of total bytes 5421
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-10b5986c] node2 
2020-12-05 09:45:02,599 AsyncStreamingInputPlus.java:197 - Issueing a channel 
read.
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.2:7012-61a7772e] node1 
2020-12-05 09:45:02,924 StreamingInboundHandler.java:187 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: 61a7772e] Received Received 
(db5364c0-36dd-11eb-ab7f-6fd7ea8cef30, #0)
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.2:7012-61a7772e] node1 
2020-12-05 09:45:02,927 StreamTransferTask.java:95 - recevied sequenceNumber 0, 
remaining files [1, 2, 3, 4]
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.2:7012-61a7772e] node1 
2020-12-05 09:45:02,929 AsyncStreamingInputPlus.java:197 - Issueing a channel 
read.
[junit-timeout] DEBUG [Streaming-EventLoop-4-1] node2 2020-12-05 09:45:10,299 
NettyStreamingMessageSender.java:258 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: acbcbd5e] Sending keep-alive
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.2:7012-61a7772e] node1 
2020-12-05 09:45:10,532 StreamingInboundHandler.java:179 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: 61a7772e] Received keep-alive
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.2:7012-61a7772e] node1 
2020-12-05 09:45:10,534 AsyncStreamingInputPlus.java:197 - Issueing a channel 
read.
[junit-timeout] DEBUG [node1_Messaging-EventLoop-3-7] node1 2020-12-05 
09:45:11,169 NettyStreamingMessageSender.java:258 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: 61a7772e] Sending keep-alive
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-acbcbd5e] node2 
2020-12-05 09:45:11,386 StreamingInboundHandler.java:179 - [Stream 
#ddd04ba0-36dd-11eb-800f-855dcc8aaeeb channel: acbcbd5e] Received keep-alive
[junit-timeout] DEBUG [Stream-Deserializer-/127.0.0.1:7012-acbcbd5e] node2 
2020-12-05 09:45:11,387 AsyncStreamingInputPlus.java:197 - Issueing a channel 
read.
[junit-timeout] DEBUG [node1_NettyStreaming-Outbound-/127.0.0.2.7012:1] node1 
2020-12-05 09:45:14,167 AsyncChannelOutputPlus.java:165 - flush took micros: 
302096082
[junit-timeout] ERROR 09:45:14 [Stream #ddd04ba0-36dd-11eb-800f-855dcc8aaeeb] 
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:208)
[junit-timeout] at 
org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:166)

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

2020-12-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-16143:
---

I can pick up review Monday unless enough committers +1 by then.

> 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: 1.5h
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]

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

2020-12-03 Thread Benjamin Lerer (Jira)


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

Benjamin Lerer commented on CASSANDRA-16143:


That sound like a good compromise to me.

> 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: 1.5h
>  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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 

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

2020-12-03 Thread Jon Meredith (Jira)


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

Jon Meredith commented on CASSANDRA-16143:
--

Bringing the discussion on what the default should be set to back here.

 

I'm fine with zero, however given Yifan added the log message to explain to 
operators why streaming failed I think they have a fighting chance of tracking 
down failures and think something like 5 minutes would provide for faster 
failure than system default TCP timeouts while tolerating the majority of messy 
server tuning like the cluster I found this on.

> 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: 1.5h
>  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 
> 

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

2020-12-01 Thread Yifan Cai (Jira)


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

Yifan Cai commented on CASSANDRA-16143:
---

Thanks [~aholmber]. So the bot links the PR, if its title starts with the 
ticket number. 

> 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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> 

[jira] [Commented] (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=17241905#comment-17241905
 ] 

Adam Holmberg commented on CASSANDRA-16143:
---

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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 

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

2020-11-16 Thread Yifan Cai (Jira)


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

Yifan Cai commented on CASSANDRA-16143:
---

Talked with Jon on Slack. Assigning it to me. 

> 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: Jon Meredith
>Priority: Normal
> Fix For: 4.0-beta
>
>
> 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) 
> [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>  [?:?]
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>  [?:?]
>at 
> 

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

2020-11-16 Thread Yifan Cai (Jira)


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

Yifan Cai commented on CASSANDRA-16143:
---

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(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}

> 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: Jon Meredith
>Priority: Normal
> Fix For: 4.0-beta
>
>
> 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