cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefania (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close
Date Fri, 13 Nov 2015 06:36:11 GMT

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

Stefania commented on CASSANDRA-10534:
--------------------------------------

Thank you for the review and for checking the remaining components. I agree with your analysis.


These are the components defined in {{SSTableWriter.components()}}:

|| Component || Notes |
| Component.DATA | Sync-ed, SequentialWriter | 
| Component.PRIMARY_INDEX  | Sync-ed, SequentialWriter |
| Component.STATS | Sync-ed, SequentialWriter |
| Component.SUMMARY | {{SSTableReader.saveSummary()}}, called in finish, not sync-ed but we
write a magic number at the end and we regenerate the summary when loading it if we don't
find this magic number, |
| Component.TOC | Not sync-ed but it's read only by standalone tools | 
| Component.DIGEST | Written by {{DataIntegrityMetadata.ChecksumWriter}}, not sync-ed and
not used but intended for users so they can validate uncompressed data files via sha1sum.
In 2.2 this becomes the adler32 checksum that can be verified with nodetool verify or the
standalone verifier.|
| Component.FILTER | Written and sync-ed manually in {{IndexWrit    er.close()}} |
| Component.COMPRESSION_INFO | To be sync-ed by this patch in {{ompressionMetadata.Writer.close()}}
|
| Component.CRC | Sync-ed, SequentialWriter |

bq. I can't quite tell since it looks like it can throw and cause other code not to execute.

Yes but this was improved in 2.2 with {{LifecycleTransaction}} and in 3.0 even further with
the removal of unfinished left overs via {{LogTransaction}}.

So, IMO, we could have problems with standalone tools and we should probably sync TOC and
DIGEST at some point but it is not critical and probaly best addressed in another ticket.
Shall I open one?

> CompressionInfo not being fsynced on close
> ------------------------------------------
>
>                 Key: CASSANDRA-10534
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10534
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sharvanath Pathak
>            Assignee: Stefania
>             Fix For: 2.1.x
>
>
> I was seeing SSTable corruption due to a CompressionInfo.db file of size 0, this happened
multiple times in our testing with hard node reboots. After some investigation it seems like
these file is not being fsynced, and that can potentially lead to data corruption. I am working
with version 2.1.9.
> I checked for fsync calls using strace, and found them happening for all but the following
components: CompressionInfo, TOC.txt and digest.sha1. All of these but the CompressionInfo
seem tolerable. Also a quick look through the code did not reveal any fsync calls. Moreover,
I suspect the commit  4e95953f29d89a441dfe06d3f0393ed7dd8586df (https://github.com/apache/cassandra/commit/4e95953f29d89a441dfe06d3f0393ed7dd8586df#diff-b7e48a1398e39a936c11d0397d5d1966R344)
has caused the regression, which removed the line
> {noformat}
>  getChannel().force(true);
> {noformat}
> from CompressionMetadata.Writer.close.
> Following is the trace I saw in system.log:
> {noformat}
> INFO  [SSTableBatchOpen:1] 2015-09-29 19:24:39,170 SSTableReader.java:478 - Opening /var/lib/cassandra/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-ka-13368
(79 bytes)
> ERROR [SSTableBatchOpen:1] 2015-09-29 19:24:39,177 FileUtils.java:447 - Exiting forcefully
due to file system exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.EOFException
>         at org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:131)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:85)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.util.CompressedSegmentedFile$Builder.metadata(CompressedSegmentedFile.java:79)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:72)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.util.SegmentedFile$Builder.complete(SegmentedFile.java:168)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:752)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:703)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:491)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:387)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:534)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_80]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) [na:1.7.0_80]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[na:1.7.0_80]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_80]
>         at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80]
> Caused by: java.io.EOFException: null
>         at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340) ~[na:1.7.0_80]
>         at java.io.DataInputStream.readUTF(DataInputStream.java:589) ~[na:1.7.0_80]
>         at java.io.DataInputStream.readUTF(DataInputStream.java:564) ~[na:1.7.0_80]
>         at org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:106)
~[apache-cassandra-2.1.9.jar:2.1.9]
>         ... 14 common frames omitted
> {noformat}
> Following is the result of ls on the data directory of a corrupted SSTable after the
hard reboot:
> {noformat}
> $ ls -l /var/lib/cassandra/data/system/sstable_activity-5a1ff267ace03f128563cfae6103c65e/
> total 60
> -rw-r--r-- 1 cassandra cassandra     0 Oct 15 09:31 system-sstable_activity-ka-1-CompressionInfo.db
> -rw-r--r-- 1 cassandra cassandra  9740 Oct 15 09:31 system-sstable_activity-ka-1-Data.db
> -rw-r--r-- 1 cassandra cassandra     0 Oct 15 09:31 system-sstable_activity-ka-1-Digest.sha1
> -rw-r--r-- 1 cassandra cassandra   880 Oct 15 09:31 system-sstable_activity-ka-1-Filter.db
> -rw-r--r-- 1 cassandra cassandra 34000 Oct 15 09:31 system-sstable_activity-ka-1-Index.db
> -rw-r--r-- 1 cassandra cassandra  7338 Oct 15 09:31 system-sstable_activity-ka-1-Statistics.db
> -rw-r--r-- 1 cassandra cassandra     0 Oct 15 09:31 system-sstable_activity-ka-1-TOC.txt
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message