Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1DF78200CF3 for ; Tue, 29 Aug 2017 14:10:12 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 1C3F81667CE; Tue, 29 Aug 2017 12:10:12 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 625A51667CC for ; Tue, 29 Aug 2017 14:10:11 +0200 (CEST) Received: (qmail 45520 invoked by uid 500); 29 Aug 2017 12:10:10 -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 45509 invoked by uid 99); 29 Aug 2017 12:10:10 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Aug 2017 12:10:10 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 12662C227F for ; Tue, 29 Aug 2017 12:10:10 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.502 X-Spam-Level: X-Spam-Status: No, score=-99.502 tagged_above=-999 required=6.31 tests=[KAM_NUMSUBJECT=0.5, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 2Ps1TirntLgL for ; Tue, 29 Aug 2017 12:10:09 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 0481660DA9 for ; Tue, 29 Aug 2017 12:10:09 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 2E423E0ED9 for ; Tue, 29 Aug 2017 12:10:06 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 3950B24167 for ; Tue, 29 Aug 2017 12:10:03 +0000 (UTC) Date: Tue, 29 Aug 2017 12:10:03 +0000 (UTC) From: "Marcus Eriksson (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11 MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 29 Aug 2017 12:10:12 -0000 [ https://issues.apache.org/jira/browse/CASSANDRA-13752?page=3Dcom.atla= ssian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId= =3D16145183#comment-16145183 ]=20 Marcus Eriksson commented on CASSANDRA-13752: --------------------------------------------- So this is because we reuse the same StreamingHistogram when we [open sstab= les early|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/= org/apache/cassandra/io/sstable/metadata/MetadataCollector.java#L292] - the= early opened sstable will be using the streaming histogram that compaction= is still building. Since CASSANDRA-13038 we can modify the contents of the= StreamingHistogram when we call {{sum()}} ({{spool}} might be compacted in= to {{bin}}). So, if someone calls the {{ColumnFamilyStoreMBean#getDroppable= TombstoneRatio}} at the wrong time we could get either the CME from CASSAND= RA-13756 or this corruption. Making StreamingHistogram thread safe is one way of fixing this, but I woul= d argue that we should be using a "builder" for StreamingHistogram - we sho= uld never access the SH while building it and for early opened sstables we = should call {{.build()}} on the StreamingHistogramBuilder and get a copy of= the internal state. Also, we should not query LIVE sstables [here|https://github.com/apache/cas= sandra/blob/cassandra-3.11/src/java/org/apache/cassandra/db/ColumnFamilySto= re.java#L2589] - it should be using {{SSTableSet.CANONICAL}} (this is proba= bly enough to fix this for now - this is the only way I can see that we acc= ess the sstablemetadata in early opened sstables). > Corrupted SSTables created in 3.11 > ---------------------------------- > > Key: CASSANDRA-13752 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1375= 2 > Project: Cassandra > Issue Type: Bug > Reporter: Hannu Kr=C3=B6ger > Assignee: Hannu Kr=C3=B6ger > Priority: Blocker > Fix For: 3.11.1 > > > We have discovered issues with corrupted SSTables.=20 > {code} > ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:57= 7 - Cannot read sstable /cassandra/data/mykeyspace/mytable-7a4992800d5611e7= b782cb90016f2d17/mc-35556-big=3D[Data.db, Statistics.db, Summary.db, Digest= .crc32, CompressionInfo.db, TOC.txt, Index.db, Filter.db]; other IO error, = skipping table > java.io.EOFException: EOF after 1898 bytes out of 21093 > at org.apache.cassandra.io.util.RebufferingInputStream.readFully(= RebufferingInputStream.java:68) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.io.util.RebufferingInputStream.readFully(= RebufferingInputStream.java:60) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.= java:402) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(= ByteBufferUtil.java:377) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMe= tadataSerializer.deserialize(StatsMetadata.java:325) ~[apache-cassandra-3.1= 1.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMe= tadataSerializer.deserialize(StatsMetadata.java:231) ~[apache-cassandra-3.1= 1.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.de= serialize(MetadataSerializer.java:122) ~[apache-cassandra-3.11.0.jar:3.11.0= ] > at org.apache.cassandra.io.sstable.metadata.MetadataSerializer.de= serialize(MetadataSerializer.java:93) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTa= bleReader.java:488) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTa= bleReader.java:396) ~[apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SST= ableReader.java:561) ~[apache-cassandra-3.11.0.jar:3.11.0] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.= java:511) [na:1.8.0_111] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1= .8.0_111] > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolEx= ecutor.java:1142) [na:1.8.0_111] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolE= xecutor.java:617) [na:1.8.0_111] > at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$thre= adLocalDeallocator$0(NamedThreadFactory.java:81) [apache-cassandra-3.11.0.j= ar:3.11.0] > {code} > Files look like this: > {code} > -rw-r--r--. 1 cassandra cassandra 3899251 Aug 7 08:37 mc-6166-big-Co= mpressionInfo.db > -rw-r--r--. 1 cassandra cassandra 16874421686 Aug 7 08:37 mc-6166-big-Da= ta.db > -rw-r--r--. 1 cassandra cassandra 10 Aug 7 08:37 mc-6166-big-Di= gest.crc32 > -rw-r--r--. 1 cassandra cassandra 2930904 Aug 7 08:37 mc-6166-big-Fi= lter.db > -rw-r--r--. 1 cassandra cassandra 111175880 Aug 7 08:37 mc-6166-big-In= dex.db > -rw-r--r--. 1 cassandra cassandra 13762 Aug 7 08:37 mc-6166-big-St= atistics.db > -rw-r--r--. 1 cassandra cassandra 882008 Aug 7 08:37 mc-6166-big-Su= mmary.db > -rw-r--r--. 1 cassandra cassandra 92 Aug 7 08:37 mc-6166-big-TO= C.txt > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org For additional commands, e-mail: commits-help@cassandra.apache.org