You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "ZhaoYang (Jira)" <ji...@apache.org> on 2020/08/11 09:48:00 UTC

[jira] [Comment Edited] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure

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

ZhaoYang edited comment on CASSANDRA-15861 at 8/11/20, 9:47 AM:
----------------------------------------------------------------

bq. 1) Orphaned hard links need to be cleaned up on startup.

If the hard links end with `.tmp`, they will be cleaned up on startup by {{StartupChecks#checkSystemKeyspaceState}}

bq. 2) Using the streaming session id for the hard link name, instead of a time uuid, would make debugging some issues easier.

+1

bq. We could leave ComponentManifest the way it was before this patch and have a separate class, let's call it ComponentContext, that embeds it.

+1

bq. In this case, if you could guarantee that no more than 1 index resample can happen at once for a given sstable, the only thing you'd need to synchronize in `cloneWithNewSummarySamplingLevel` is `saveSummary`. If you did that, you could just synchronize hard link creation on `tidy.global`, instead of introducing a new lock.

Agreed with caleb, no more than 1 index resample can happen concurrently for a given sstable as sstable is marked as compacting before resampling.

bq. That leaves indexSummary, which perhaps we cold make volatile, and all the state used in cloneAndReplace()...but we could just extend the synchronized (tidy.global) block to include the latter. Nothing expensive happens inside cloneAndReplace(), AFAICT.

good idea

bq. synchronized (tidy.global)

The old approach was to synchronized entire streaming phase, so I didn't use "synchronized (tidy.global)" which may block concurrent compactions. 

But now only hard-link creation is synchronized, using "synchronized (tidy.global)" is better than introducing a new lock.



was (Author: jasonstack):
bq. 1) Orphaned hard links need to be cleaned up on startup.

If the hard links end with `.tmp`, they will be cleaned up on startup by {{StartupChecks#checkSystemKeyspaceState}}

bq. 2) Using the streaming session id for the hard link name, instead of a time uuid, would make debugging some issues easier.

+1

bq. We could leave ComponentManifest the way it was before this patch and have a separate class, let's call it ComponentContext, that embeds it.

+1

bq. In this case, if you could guarantee that no more than 1 index resample can happen at once for a given sstable, the only thing you'd need to synchronize in `cloneWithNewSummarySamplingLevel` is `saveSummary`. If you did that, you could just synchronize hard link creation on `tidy.global`, instead of introducing a new lock.

Agreed with caleb, no more than 1 index resample can happen concurrently for a given sstable as sstable is marked as compacting before resampling.

bq. That leaves indexSummary, which perhaps we cold make volatile, and all the state used in cloneAndReplace()...but we could just extend the synchronized (tidy.global) block to include the latter. Nothing expensive happens inside cloneAndReplace(), AFAICT.

good idea

bq. synchronized (tidy.global)




> 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: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org