[
https://issues.apache.org/jira/browse/CASSANDRA-10534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sharvanath Pathak updated CASSANDRA-10534:
------------------------------------------
Description:
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 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
{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}
was:
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 seem
tolerable but the CompressionInfo seem tolerable. Also a quick look through
the code and did not revealed 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
{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}
> CompressionInfo not being fsynced on close
> ------------------------------------------
>
> Key: CASSANDRA-10534
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10534
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Sharvanath Pathak
> 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 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
> {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}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)