*Description*
When adding a new node to an existing cluster, the new node
bootstrapping fails with the
"io.netty.channel.unix.Errors$NativeIoException: writeAddress(..)
failed: Connection timed out" error from the streaming source node.
Resuming the bootstrap with "nodetool bootstrap resume" works, but the
resumed bootstrap can fail too. We often need to run "nodetool bootstrap
resume" a couple of times to complete the bootstrapping on a joining node.
Steps that produced the error
(I'm hesitant to say "step to reproduce", because I failed to reproduce
the error on a testing cluster)
Install Cassandra 4.1.1 on new servers, using two of the existing nodes
as seed nodes, start the new node and let it join the cluster. Watch the
logs.
Environment
All nodes, existing or new, have the same software versions as below.
Cassandra: version 4.1.1
Java: OpenJDK 11
OS: Debian 11
Existing nodes each has 1TB SSD, 64GB memory and 6 cores CPU, and
num_tokens is set to 4
New nodes each has 2TB SSD, 128GB memory and 16 cores CPU, and
num_tokens is set to 8
Cassandra is in a single DC, single rack setup with about 130 nodes, and
all non-system keyspaces have RF=3
Relevant config options:
stream_throughput_outbound: 15MiB/s
streaming_connections_per_host: 2
auto_bootstrap: not set, default to true
internode_tcp_user_timeout: not set, default to 30 seconds
internode_streaming_tcp_user_timeout: not set, default to 5 minutes
streaming_keep_alive_period: not set, default to 5 minutes
streaming_state_expires: not set, default to 3 days
streaming_state_size: not set, default to 40MiB
streaming_stats_enabled: not set, default to true
uuid_sstable_identifiers_enabled: true (turned on after upgraded
to 4.1 last year)
What we have tried
*Tried*: checking the hardware and network
*Result*: everything appears to be fine
*Tried*: Google searching for the error message
"io.netty.channel.unix.Errors$NativeIoException: writeAddress(..)
failed: Connection timed out"
*Result*: only one matching result was found, and it points to
CASSANDRA-16143 <https://issues.apache.org/jira/browse/CASSANDRA-16143>.
That certainly doesn't apply in our case, as it was fixed in 4.0, and I
also don't believe our data centre grade SSDs are that slow.
*Tried*: reducing the stream_throughput_outbound from 30 to 15 MiB/s
*Result*: did not help, no sign of any improvement
*Tried*: analyse the logs from the joining node and the streaming source
nodes
*Result*: the error says the write connection timed out on the sending
end, but a few seconds before that, both sending and receiving ends of
the connection were still communicating with each other. I couldn't make
sense of it.
*Tried*: bootstrapping a different node of the same spec
*Result*: same error reproduced
*Tried*: attempting to reproduce the error on a testing cluster
*Result*: unable to reproduce this error on a smaller testing cluster
with less nodes, less powerful hardware, same Cassandar, Java and OS
version, same config, same schema, less data and same mixed number of
vnodes.
*Tried*: keep retrying with "nodetool bootstrap resume"
*Result*: this works and unblocked us from adding new nodes to the
cluster, but this obviously is not how it should be done.
What do I expect from posting this
I'm suspecting that this is a bug in Cassandra, but lack the evidence to
support that, and lacks the expertise in debugging Cassandra (or any
other Java application).
It would be much appreciated if anyone could offer me some help on this,
or point me to a direction that may lead to the solution.
Relevant logs
Note: IP address, keyspace and table names are reducted. The IP address
ending in 111 is the joining node, and the IP address ending in 182 was
one of the streaming source node.
The logs from the joining node (IP: xxx.xxx.xxx.111):
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,555 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: e0e09450] Received
Prepare SYNACK ( 440 files}
INFO [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,556 StreamResultFuture.java:187 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e ID#0] Prepare completed.
Receiving 440 files(38.941GiB), sending 0 files(0.000KiB)
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,556 StreamCoordinator.java:148 - Connecting next
session 69de5e80-4f21-11ee-abc5-1de0bb481b0e with /95.217.36.91:7000.
INFO [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,556 StreamSession.java:368 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Starting streaming to
95.217.36.91:7000
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,556 StreamingMultiplexedChannel.java:167 -
Creating stream session to /95.217.36.91:7000 as initiator
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,569 StreamingMultiplexedChannel.java:471 -
[Stream #69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: 5612da62]
Scheduling keep-alive task with 300s period.
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,569 StreamingMultiplexedChannel.java:178 -
Creating control channel.remote 95.217.36.91/95.217.36.91:7000
channel.local /xxx.xxx.xxx.111:40124 channel.id 5612da62
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,569 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
INITIALIZED to PREPARING
INFO [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,569 StreamCoordinator.java:268 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e, ID#0] Beginning stream
session with /95.217.36.91:7000
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 15:59:13,569 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
PREPARING to STREAMING
INFO [Messaging-EventLoop-3-27] 2023-09-09 15:59:13,569
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54860)->/xxx.xxx.xxx.111:7000-STREAMING-3de3a307
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-29] 2023-09-09 15:59:13,570
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54886)->/xxx.xxx.xxx.111:7000-STREAMING-a892c215
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-28] 2023-09-09 15:59:13,570
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54876)->/xxx.xxx.xxx.111:7000-STREAMING-73f7fc4b
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-26] 2023-09-09 15:59:13,570
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54844)->/xxx.xxx.xxx.111:7000-STREAMING-d1df2b4c
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-3] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54938)->/xxx.xxx.xxx.111:7000-STREAMING-b5e466f3
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-4] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54940)->/xxx.xxx.xxx.111:7000-STREAMING-95d05ed6
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-5] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54942)->/xxx.xxx.xxx.111:7000-STREAMING-7f1fd046
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-31] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54902)->/xxx.xxx.xxx.111:7000-STREAMING-109d9812
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-1] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54922)->/xxx.xxx.xxx.111:7000-STREAMING-9c2bdad1
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-2] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54906)->/xxx.xxx.xxx.111:7000-STREAMING-45cdfbbc
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-32] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54904)->/xxx.xxx.xxx.111:7000-STREAMING-52b56ced
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
INFO [Messaging-EventLoop-3-30] 2023-09-09 15:59:13,571
InboundConnectionInitiator.java:465 -
/xxx.xxx.xxx.182:7000(/xxx.xxx.xxx.182:54888)->/xxx.xxx.xxx.111:7000-STREAMING-a4a2a432
streaming connection established, version = 12, framing =
UNPROTECTED, encryption = unencrypted
====================================================
...... many lines of logs, everything looks fine
...... about 7 minutes later
====================================================
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:41,842 CassandraIncomingFile.java:69 - Incoming
stream entireSSTable=false components=null
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:41,842 CassandraCompressedStreamReader.java:69 -
[Stream #69de5e80-4f21-11ee-abc5-1de0bb481b0e] Start receiving file
#3 from /xxx.xxx.xxx.182:7000, repairedAt = 1694218491888, size =
47803, ks = 'ksname', pendingRepair = 'null', table = 'tablename'.
INFO [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:41,995 CassandraCompressedStreamReader.java:105 -
[Stream #69de5e80-4f21-11ee-abc5-1de0bb481b0e] Finished receiving
file #3 from /xxx.xxx.xxx.182:7000 readBytes = 46.683KiB, totalSize
= 46.683KiB
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:41,996 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: 45cdfbbc] Received
IncomingStreamMessage{header=Header (tableId:
a1d2d2a0-59e2-11e9-b864-5d754f83b3f3, #3, repairedAt: 1694218491888,
pendingRepair: null, sendByFollower: true),
stream=CassandraIncomingFile{sstable=ksname/tablename}}
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:41,996 StreamReceiveTask.java:87 - received 3 of 9
total files, 5422378 of total bytes 47803
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:42,487 CassandraIncomingFile.java:69 - Incoming
stream entireSSTable=false components=null
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-45cdfbbc]
2023-09-09 16:06:42,487 CassandraCompressedStreamReader.java:69 -
[Stream #69de5e80-4f21-11ee-abc5-1de0bb481b0e] Start receiving file
#4 from /xxx.xxx.xxx.182:7000, repairedAt = 1679888028671, size =
1160433119, ks = 'ksname', pendingRepair = 'null', table = 'tablename'.
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 16:06:43,708 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: e0e09450] Received
Session Failed
ERROR [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 16:06:43,708 StreamSession.java:930 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Remote peer
/xxx.xxx.xxx.182:7000 failed stream session.
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.182:7000-e0e09450]
2023-09-09 16:06:43,708 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
STREAMING to FAILED
The logs from one of the existing node (IP: xxx.xxx.xxx.182), which was
the first existing node to encounter an error during the streaming:
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 15:59:13,555 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: a8cca64c] Received
Prepare ACK
INFO [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 15:59:13,556 StreamResultFuture.java:187 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e ID#0] Prepare completed.
Receiving 0 files(0.000KiB), sending 440 files(38.941GiB)
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 15:59:13,556 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
PREPARING to STREAMING
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:1] 2023-09-09
15:59:13,556 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:2] 2023-09-09
15:59:13,556 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:3] 2023-09-09
15:59:13,556 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:4] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:5] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:6] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:7] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:9] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:10] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:11] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:12] 2023-09-09
15:59:13,557 StreamingMultiplexedChannel.java:184 - Creating stream
session to /xxx.xxx.xxx.111:7000 as follower
====================================================
...... many lines of logs, everything looks fine
...... about 7 minutes later
====================================================
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:33,237 CassandraCompressedStreamWriter.java:101 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Finished streaming file
/var/lib/cassandra/data/ksname/tablename-a1d2d2a059e211e9b8645d754f83b3f3/nb-3g98_020l_1ffvi1yyu5urthapoa-big-Data.db
to /xxx.xxx.xxx.111:7000, bytesTransferred = 46.683KiB, totalSize =
46.683KiB
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 16:06:41,836 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: a8cca64c] Received
Received (a1d2d2a0-59e2-11e9-b864-5d754f83b3f3, #2)
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 16:06:41,836 StreamTransferTask.java:93 - received
sequenceNumber 2, remaining files [0, 3, 4, 5, 6, 7, 8]
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 16:06:41,996 StreamDeserializingTask.java:74 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e channel: a8cca64c] Received
Received (a1d2d2a0-59e2-11e9-b864-5d754f83b3f3, #3)
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 16:06:41,996 StreamTransferTask.java:93 - received
sequenceNumber 3, remaining files [0, 4, 5, 6, 7, 8]
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:42,141 CassandraCompressedStreamWriter.java:62 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Start streaming file
/var/lib/cassandra/data/ksname/tablename-a1d2d2a059e211e9b8645d754f83b3f3/nb-33045-big-Data.db
to /xxx.xxx.xxx.111:7000, repairedAt = 1679888028671, totalSize =
1160433119
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:42,141 CassandraCompressedStreamWriter.java:80 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Writing section 0 with length
1160433119 to stream.
ERROR [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,701 StreamSession.java:696 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Streaming error occurred on
session with peer xxx.xxx.xxx.111:7000 through xxx.xxx.xxx.111:42718
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:124)
at
org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:90)
at
org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:120)
at
org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:89)
at
org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:179)
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:39)
at
org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:318)
at
org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81)
at
org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47)
at
org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
Suppressed: java.nio.channels.ClosedChannelException: null
at
org.apache.cassandra.net.AsyncStreamingOutputPlus.doFlush(AsyncStreamingOutputPlus.java:82)
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.NettyStreamingChannel$1.close(NettyStreamingChannel.java:141)
at
org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:319)
... 7 common frames omitted
Caused by: io.netty.channel.unix.Errors$NativeIoException:
writeAddress(..) failed: Connection timed out
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,702 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
STREAMING to FAILED
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,702 StreamSession.java:551 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Changing session state from
FAILED to FAILED
DEBUG [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,702 StreamSession.java:518 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Will close attached inbound
{a8cca64c=org.apache.cassandra.streaming.async.NettyStreamingChannel@5e2750e5}
and outbound
{082e1eb5=org.apache.cassandra.streaming.async.NettyStreamingChannel@1a43113d,
0a5d0399=org.apache.cassandra.streaming.async.NettyStreamingChannel@66f1d031,
c0a46d3c=org.apache.cassandra.streaming.async.NettyStreamingChannel@6e576202,
449a96c7=org.apache.cassandra.streaming.async.NettyStreamingChannel@a13a755,
7e8611b8=org.apache.cassandra.streaming.async.NettyStreamingChannel@8db95d3,
afc5031b=org.apache.cassandra.streaming.async.NettyStreamingChannel@60c3bca1,
3e1bde17=org.apache.cassandra.streaming.async.NettyStreamingChannel@35d185c9,
47d691ef=org.apache.cassandra.streaming.async.NettyStreamingChannel@226fa529,
f28c903b=org.apache.cassandra.streaming.async.NettyStreamingChannel@76cec9ba,
07e63dd0=org.apache.cassandra.streaming.async.NettyStreamingChannel@b955576,
bb34c8b3=org.apache.cassandra.streaming.async.NettyStreamingChannel@32d36600}
channels
DEBUG [Stream-Deserializer-/xxx.xxx.xxx.111:7000-a8cca64c]
2023-09-09 16:06:43,702 StreamSession.java:657 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Socket closed after session
completed with state FAILED
DEBUG [Messaging-EventLoop-3-5] 2023-09-09 16:06:43,702
StreamingMultiplexedChannel.java:512 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Closing stream connection
channels on /xxx.xxx.xxx.111:7000
INFO [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,702 StreamResultFuture.java:201 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Session with
/xxx.xxx.xxx.111:7000 is failed
WARN [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:8] 2023-09-09
16:06:43,702 StreamResultFuture.java:242 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Stream failed
ERROR [NettyStreaming-Outbound-/xxx.xxx.xxx.111.7000:11] 2023-09-09
16:06:43,702 StreamSession.java:696 - [Stream
#69de5e80-4f21-11ee-abc5-1de0bb481b0e] Streaming error occurred on
session with peer xxx.xxx.xxx.111:7000 through xxx.xxx.xxx.111:42718
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:124)
at
org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:90)
at
org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:120)
at
org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:89)
at
org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:179)
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:39)
at
org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:318)
at
org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81)
at
org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47)
at
org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
Suppressed: java.nio.channels.ClosedChannelException: null
at
org.apache.cassandra.net.AsyncStreamingOutputPlus.doFlush(AsyncStreamingOutputPlus.java:82)
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.NettyStreamingChannel$1.close(NettyStreamingChannel.java:141)
at
org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:319)
... 7 common frames omitted
Caused by: io.netty.channel.StacklessClosedChannelException: null
at
io.netty.channel.AbstractChannel.close(ChannelPromise)(Unknown Source)
====================================================
...... more lines like above for other outbound streaming connections
====================================================