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

Caleb Rackliffe commented on CASSANDRA-15861:
---------------------------------------------

bq. During entire-sstable streaming, CassandraOutgoingFile will execute the 
streaming code within the sstable read-lock. So multiple streamings on the same 
sstable can start at the same time. I think it's fine to block 
stats-mutation/index-summary redistribution until streaming completion.

I'm still looking at the patch itself, but the thing I'm most curious about at 
a design level is whether holding the SSTable read lock until we write the 
entirety of the SSTable to the network is safe in the face of a concurrent 
incremental repair. In other words, how likely is it that an incremental repair 
already in flight would have to wait a significant period of time to complete 
with an entire SSTable in the middle of streaming?

We've already talked about the possibility of just buffering the stats 
component, but if we think that makes things too hard to reason around, what if 
we instead broke the streaming into mutable vs. immutable components and wrote 
the mutable components first? That might at least reduce the window, and repair 
couldn't block while the largest components (like the data file, and perhaps 
index components in the future) are streaming.

WDYT?

CC [~bdeggleston]

> Mutating sstable component may race with entire-sstable-streaming(ZCS) 
> causing checksum validation failure
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-15861
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15861
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Consistency/Repair, Consistency/Streaming, 
> Local/Compaction
>            Reporter: ZhaoYang
>            Assignee: ZhaoYang
>            Priority: Normal
>             Fix For: 4.0-beta
>
>
> Flaky dtest: [test_dead_sync_initiator - 
> repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/]
> {code:java|title=stacktrace}
> Unexpected error found in node logs (see stdout for full details). Errors: 
> [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 
> CassandraEntireSSTableStreamReader.java:145 - [Stream 
> 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream 
> for table = keyspace1.standard1
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db
>       at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219)
>       at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198)
>       at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129)
>       at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226)
>       at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140)
>       at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78)
>       at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:49)
>       at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:36)
>       at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:49)
>       at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:181)
>       at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.IOException: Checksums do not match for 
> /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db
> {code}
>  
> In the above test, it executes "nodetool repair" on node1 and kills node2 
> during repair. At the end, node3 reports checksum validation failure on 
> sstable transferred from node1.
> {code:java|title=what happened}
> 1. When repair started on node1, it performs anti-compaction which modifies 
> sstable's repairAt to 0 and pending repair id to session-id.
> 2. Then node1 creates {{ComponentManifest}} which contains file lengths to be 
> transferred to node3.
> 3. Before node1 actually sends the files to node3, node2 is killed and node1 
> starts to broadcast repair-failure-message to all participants in 
> {{CoordinatorSession#fail}}
> 4. Node1 receives its own repair-failure-message and fails its local repair 
> sessions at {{LocalSessions#failSession}} which triggers async background 
> compaction.
> 5. Node1's background compaction will mutate sstable's repairAt to 0 and 
> pending repair id to null via  
> {{PendingRepairManager#getNextRepairFinishedTask}}, as there is no more 
> in-progress repair.
> 6. Node1 actually sends the sstable to node3 where the sstable's STATS 
> component size is different from the original size recorded in the manifest.
> 7. At the end, node3 reports checksum validation failure when it tries to 
> mutate sstable level and "isTransient" attribute in 
> {{CassandraEntireSSTableStreamReader#read}}.
> {code}
> Currently, entire-sstable-streaming requires sstable components to be 
> immutable, because \{{ComponentManifest}}
> with component sizes are sent before sending actual files. This isn't a 
> problem in legacy streaming as STATS file length didn't matter.
>  
> Ideally it will be great to make sstable STATS metadata immutable, just like 
> other sstable components, so we don't have to worry this special case.
> I can think of 2 ways:
>  # Make STATS mutation as a proper compaction to create hard link on the 
> compacting sstable components with a new descriptor, except STATS files which 
> will be copied entirely. Then mutation will be applied on the new STATS file. 
> At the end, old sstable will be released. This ensures all sstable components 
> are immutable and shouldn't make these special compaction tasks slower.
>  # Change STATS metadata format to use fixed length encoding for repair info



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to