From commits-return-237493-archive-asf-public=cust-asf.ponee.io@cassandra.apache.org Tue Sep 1 06:22:05 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mxout1-he-de.apache.org (mxout1-he-de.apache.org [95.216.194.37]) by mx-eu-01.ponee.io (Postfix) with ESMTPS id 7493D180660 for ; Tue, 1 Sep 2020 08:22:05 +0200 (CEST) Received: from mail.apache.org (mailroute1-lw-us.apache.org [207.244.88.153]) by mxout1-he-de.apache.org (ASF Mail Server at mxout1-he-de.apache.org) with SMTP id DC1D56358C for ; Tue, 1 Sep 2020 06:22:04 +0000 (UTC) Received: (qmail 80241 invoked by uid 500); 1 Sep 2020 06:22:03 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 80199 invoked by uid 99); 1 Sep 2020 06:22:02 -0000 Received: from mailrelay1-us-west.apache.org (HELO mailrelay1-us-west.apache.org) (209.188.14.139) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 01 Sep 2020 06:22:02 +0000 Received: from jira-he-de.apache.org (static.172.67.40.188.clients.your-server.de [188.40.67.172]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 91E4D43E5A for ; Tue, 1 Sep 2020 06:22:01 +0000 (UTC) Received: from jira-he-de.apache.org (localhost.localdomain [127.0.0.1]) by jira-he-de.apache.org (ASF Mail Server at jira-he-de.apache.org) with ESMTP id 467C9780CAC for ; Tue, 1 Sep 2020 06:22:00 +0000 (UTC) Date: Tue, 1 Sep 2020 06:22:00 +0000 (UTC) From: "Stefan Miklosovic (Jira)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=3Dcom.atla= ssian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId= =3D17188162#comment-17188162 ]=20 Stefan Miklosovic edited comment on CASSANDRA-15861 at 9/1/20, 6:21 AM: ------------------------------------------------------------------------ This clashes with https://issues.apache.org/jira/browse/CASSANDRA-15406 I a= m working on for a very long time and I havent been able to manage to merge= it. There seems to be a clash with some functionality related to how size = of files is computed because it is broken and it reports wrong numbers in n= etstats. Could somebody verify that this patch is compatible with 15406 and it does = not break things? It is pretty frustrating to continuously rewrite already = fully prepared and tested code. The problem with the original code is nicely summarized in this comment dow= nwards and input from Benjamin Lerer https://issues.apache.org/jira/browse/= CASSANDRA-15406?focusedCommentId=3D17181389&page=3Dcom.atlassian.jira.plugi= n.system.issuetabpanels%3Acomment-tabpanel#comment-17181389 was (Author: stefan.miklosovic): This clashes with https://issues.apache.org/jira/browse/CASSANDRA-15406 I a= m working on for a very long time and I havent been able to manage to merge= it. There seems to be a clash with some functionality related to how size = of files is computed because it is broken and it reports wrong numbers in n= etstats. Could somebody verify that this patch is compatible with 15406 and it does = not break things? It is pretty frustrating to continuously rewrite already = fully prepared and tested code. > Mutating sstable component may race with entire-sstable-streaming(ZCS) ca= using checksum validation failure > -------------------------------------------------------------------------= --------------------------------- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1586= 1 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, Local/Com= paction > Reporter: ZhaoYang > Assignee: ZhaoYang > Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - repair_tests.repair_test.TestRep= air|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/= 143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sy= nc_initiator/] > {code:java|title=3Dstacktrace} > 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-11e= a-a808-2f23710fdc90] Error while reading sstable from stream for table =3D = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /home= /cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/ke= yspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > =09at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeVa= lidateChecksum(MetadataSerializer.java:219) > =09at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deseria= lize(MetadataSerializer.java:198) > =09at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deseria= lize(MetadataSerializer.java:129) > =09at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(= MetadataSerializer.java:226) > =09at org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReade= r.read(CassandraEntireSSTableStreamReader.java:140) > =09at org.apache.cassandra.db.streaming.CassandraIncomingFile.read(Cassan= draIncomingFile.java:78) > =09at org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.des= erialize(IncomingStreamMessage.java:49) > =09at org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.des= erialize(IncomingStreamMessage.java:36) > =09at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(S= treamMessage.java:49) > =09at org.apache.cassandra.streaming.async.StreamingInboundHandler$Stream= DeserializingTask.run(StreamingInboundHandler.java:181) > =09at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLoca= lRunnable.java:30) > =09at java.lang.Thread.run(Thread.java:748) > Caused by: java.io.IOException: Checksums do not match for /home/cassandr= a/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/s= tandard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > {code} > =C2=A0 > In the above test, it executes "nodetool repair" on node1 and kills node2= during repair. At the end, node3 reports checksum validation failure on ss= table transferred from node1. > {code:java|title=3Dwhat happened} > 1. When repair started on node1, it performs anti-compaction which modifi= es sstable's repairAt to 0 and pending repair id to session-id. > 2. Then node1 creates {{ComponentManifest}} which contains file lengths t= o be transferred to node3. > 3. Before node1 actually sends the files to node3, node2 is killed and no= de1 starts to broadcast repair-failure-message to all participants in {{Coo= rdinatorSession#fail}} > 4. Node1 receives its own repair-failure-message and fails its local repa= ir sessions at {{LocalSessions#failSession}} which triggers async backgroun= d compaction. > 5. Node1's background compaction will mutate sstable's repairAt to 0 and = pending repair id to null via {{PendingRepairManager#getNextRepairFinished= Task}}, as there is no more in-progress repair. > 6. Node1 actually sends the sstable to node3 where the sstable's STATS co= mponent 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 {{CassandraEntireSSTab= leStreamReader#read}}. > {code} > Currently, entire-sstable-streaming requires sstable components to be imm= utable, because \{{ComponentManifest}} > with component sizes are sent before sending actual files. This isn't a p= roblem in legacy streaming as STATS file length didn't matter. > =C2=A0 > Ideally it will be great to make sstable STATS metadata immutable, just l= ike 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 whi= ch 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 co= mponents are immutable and shouldn't make these special compaction tasks sl= ower. > # Change STATS metadata format to use fixed length encoding for repair i= nfo -- 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