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

David Capwell commented on CASSANDRA-17081:
-------------------------------------------

and of corse I am wrong... failed in CI for the patch in CASSANDRA-17085 (patch 
still needed to fix threads); new issue happened which was different than 
previous ones... zero-copy-streaming causes the disk failure policy to trigger 
on channel close...

{code}
ERROR [Stream-Deserializer-/127.0.0.1:7000-bcd266ec] 2021-11-02 21:49:39,859 
DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
exception on startup, disk failure policy "stop"
org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
        at 
org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
        at 
org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
        at 
org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
        at 
org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
        at 
org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
        at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
        at 
org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
        at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.nio.channels.ClosedChannelException: null
        at 
org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
        at 
org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
        at 
org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
        ... 9 common frames omitted
{code}

> Fix test: 
> bootstrap_test.py::TestBootstrap::test_bootstrap_with_reset_bootstrap_state
> -------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-17081
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17081
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Test/dtest/python
>            Reporter: Josh McKenzie
>            Assignee: David Capwell
>            Priority: Normal
>             Fix For: NA
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Seeing in circle and locally on trunk:
> Looks like it's timing out waiting for the bootstrap to complete.
> {code:java}
> test_bootstrap_with_reset_bootstrap_state failed (1 runs remaining out of 2).
>         <class 'ccmlib.node.TimeoutError'>
>         28 Oct 2021 19:03:53 [node3] after 120.39/120 seconds Missing: 
> ['127.0.0.1:7000.* is now UP'] not found in system.log:
>  Head: ERROR [Stream-Deserializer-/127.0.0.1:7000-20b885c
>  Tail: ...b336de0e72/nb-1-big-Data.db 
> ERROR [Stream-Deserializer-/127.0.0.1:7000-29a7cdb5] 2021-10-28 15:01:36,578 
> StorageService.java:483 - Stopping gossiper
>         [<TracebackEntry 
> /Users/jmckenzie/src/cassandra-dtest/bootstrap_test.py:483>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:895>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:664>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:588>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:56>]
> test_bootstrap_with_reset_bootstrap_state failed; it passed 0 out of the 
> required 1 times.
>         <class 'ccmlib.node.TimeoutError'>
>         28 Oct 2021 19:08:23 [node3] after 120.41/120 seconds Missing: 
> ['127.0.0.1:7000.* is now UP'] not found in system.log:
>  Head: 
>  Tail: ...
>         [<TracebackEntry 
> /Users/jmckenzie/src/cassandra-dtest/bootstrap_test.py:483>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:895>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:664>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:588>
> <TracebackEntry /Users/jmckenzie/src/ccm/ccmlib/node.py:56>]
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to