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

Jon Meredith commented on CASSANDRA-18736:
------------------------------------------

As expected, the (modified) tests fail on 4.0 so I've backported there too. 
I'll try and post test runs tomorrow but wanted to share the changes in case 
anybody was waiting on it.

PRs
Trunk https://github.com/apache/cassandra/pull/3250
5.0 https://github.com/apache/cassandra/pull/3251
4.1 https://github.com/apache/cassandra/pull/3252
4.0 https://github.com/apache/cassandra/pull/3253

> Streaming exception race creates corrupt transaction log files that prevent 
> restart
> -----------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-18736
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18736
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Consistency/Streaming, Local/Startup and Shutdown
>            Reporter: Jon Meredith
>            Assignee: Jon Meredith
>            Priority: Normal
>             Fix For: 4.0.x, 4.1.x, 5.0.x
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> On restart, Cassandra logs this message and terminates.
> {code:java}
> ERROR 2023-07-17T17:17:22,931 [main] 
> org.apache.cassandra.db.lifecycle.LogTransaction:561 - Unexpected disk state: 
> failed to read transaction log 
> [nb_txn_stream_39d5f6b0-fb81-11ed-8f46-e97b3f61511e.log in 
> /datadir1/keyspace/table-c9527530a0d611e8813f034699fc9043]
> Files and contents follow:
> /datadir1/keyspace/table-c9527530a0d611e8813f034699fc9043/nb_txn_stream_39d5f6b0-fb81-11ed-8f46-e97b3f61511e.log
>         ABORT:[,0,0][737437348]
>                 ***This record should have been the last one in all replicas
>         
> ADD:[/datadir1/keyspace/table-c9527530a0d611e8813f034699fc9043/nb-284490-big-,0,8][2493503833]
> {code}
> The root cause is a race during streaming exception handling.
> Although concurrent modification of to the {{LogTransaction}} was added for 
> CASSANDRA-16225, there is nothing to prevent usage after the transaction is 
> completed (committed/aborted) once it has been processed by 
> {{TransactionTidier}} (after the last reference is released). Before the 
> transaction is tidied, the {{LogFile}} keeps a list of records that are 
> checked for completion before adding new entries. In {{TransactionTidier}} 
> {{LogFile.records}} are cleared as no longer needed, however the 
> LogTransaction/LogFile is still accessible to the stream.
> The changes in CASSANDRA-17273 added a parallel set of {{onDiskRecords}} that 
> could be used to reliably recreate the transaction log at any new datadirs 
> the same as the existing
> datadirs - regardless of the effect of 
> {{LogTransaction.untrackNew/LogFile.remove}}
> If a streaming exception causes the LogTransaction to be aborted and tidied 
> just before {{SimpleSSTableMultiWriter}} calls trackNew to add a new sstable. 
> At the time of the call, the {{LogFile}} will not contain any {{LogReplicas}},
> {{LogFile.records}} will be empty, and {{LogFile.onDiskRecords}} will contain 
> an {{ABORT}}.
> When {{LogTransaction.trackNew/LogFile.add}} is called, the check for 
> completed transaction fails as records is empty, there are no replicas on the 
> datadir, so {{maybeCreateReplicas}} creates a new txnlog file replica 
> containing ABORT, then
> appends an ADD record.
> The LogFile has already been tidied after the abort so the txnlog file is not 
> removed and sits on disk until a restart, causing the faiulre.
> There is a related exception caused with a different interleaving of aborts, 
> after an sstable is added, however this is just a nuisance in the logs as the 
> LogRelica is already created with an {{ADD}} record first.
> {code:java}
> java.lang.AssertionError: 
> [ADD:[/datadir1/keyspace/table/nb-23314378-big-,0,8][1869379820]] is not 
> tracked by 55be35b0-35d1-11ee-865d-8b1e3c48ca06
>         at org.apache.cassandra.db.lifecycle.LogFile.remove(LogFile.java:388)
>         at 
> org.apache.cassandra.db.lifecycle.LogTransaction.untrackNew(LogTransaction.java:158)
>         at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.untrackNew(LifecycleTransaction.java:577)
>         at 
> org.apache.cassandra.db.streaming.CassandraStreamReceiver$1.untrackNew(CassandraStreamReceiver.java:149)
>         at 
> org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.abort(SimpleSSTableMultiWriter.java:95)
>         at 
> org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter.abort(RangeAwareSSTableWriter.java:191)
>         at 
> org.apache.cassandra.db.streaming.CassandraCompressedStreamReader.read(CassandraCompressedStreamReader.java:115)
>         at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:85)
>         at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:53)
>         at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:38)
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:53)
>         at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:172)
>         at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.base/java.lang.Thread.run(Thread.java:829)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to