[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-16 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

Created CASSANDRA-10709 and CASSANDRA-10710.

> CompressionInfo not being fsynced on close
> --
>
> Key: CASSANDRA-10534
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10534
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local Write-Read Paths
>Reporter: Sharvanath Pathak
>Assignee: Stefania
> Fix For: 2.1.12, 2.2.4, 3.0.1, 3.1
>
>
> 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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-13 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg commented on CASSANDRA-10534:


Yes please open new low priority ticket. It can't be a ton of work to sync the 
TOC and digest and it would reduce the window where they are inconsistent.

A bigger issue brought up by this is that we don't do power failure testing in 
a loop so we can't actually verify the correctness of Cassandra under those 
conditions. Can you create a major priority ticket for that?

> 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.(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.(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}

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-12 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg commented on CASSANDRA-10534:


Ok. +1 LGTM.

It looks like the other components all use SequentialWriter which if you use 
the transaction stuff does the right thing without any extra work. Except we 
don't use the transaction proxy for the other metadata components instead we 
call finish manually which goes through the transaction proxy.

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

> 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.(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.(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 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-12 Thread Stefania (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10534?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-11 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg commented on CASSANDRA-10534:


I don't see why we would ever not sync the other files? Are they really not 
necessary for the sstable to be readable? If they are required then they need 
to be synced as well otherwise we are going to take actions based on the 
sstable being durable/readable when it isn't really readable.

> 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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-11 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

_TOC.txt_ is only used by standalone tools, we do a listing of the folder in 
the CFS constructor. I could not find where the digest file is read, at least 
not on the 2.1 code. Other components I have not checked yet. For sure index 
and data files are sync-ed.

We should probably sync all sstable components that we write but should we fix 
this regression and open a new ticket for better visibility?

> 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.(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.(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 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-09 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

[~benedict], I've rebased and re-run CI on 2.1 and 2.2. It looks OK to me (a 
few failing tests but inline with the main branches). 

Let me know if there is anything else to be done before committing.

> 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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-09 Thread JIRA

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

Fredrik Larsson Stigbäck commented on CASSANDRA-10534:
--

Will this one be included in 2.1.12?

> 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.(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.(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-- 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-11-05 Thread Study Hsueh (JIRA)

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

Study Hsueh commented on CASSANDRA-10534:
-

This bug also happened in 2.1.10.

> 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.(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.(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 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-18 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

I've rebased and restarted all 3 jobs.

> 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
>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.(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.(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 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-10534:
--

In all cases we will need to call flush before calling sync, since we have a 
buffered writer. {{close}} is idempotent, so that should not be a problem.

> 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
>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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

I've added the call to {{flush}} in a separate commit. 

I tried to abort the CI jobs and restart new ones but it seems the abort only 
removed the jobs in the queue so it's the old jobs (without flush) that are 
running at the moment. If they are aborted later on I will restart them or if I 
am offline you can restart yourself with cassci {{!build 
stef1927-10534-3.0-dtest}} etc.

> 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
>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.(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.(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 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

[~benedict], [~sharvanath] analysis is correct: since CASSANDRA-6916 we no 
longer fsync compression metadata after writing it. I've attached a small 
[patch|https://github.com/stef1927/cassandra/commits/10534-2.1] that should fix 
this, can you take a look?

If the patch is fine I will run CI on the 2.1+ branches. 

> 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
>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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-10534:
--

Hi [~sharvanath]: thanks for taking the time to strace this and find our (my) 
mistake. 

[~stefania]: thanks for providing a patch. LGTM. I'll commit once we have clean 
CI results.

> 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
>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.(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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Stefania (JIRA)

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

Stefania commented on CASSANDRA-10534:
--

[~Benedict]: I merely wanted to fsync in case of partial write but if it looks 
too unusual we can have it in the try block. I amended the commit and force 
pushed.  The 2.2 patch is a rewrite because the code is too divergent. I 
believe the FOS close is idempotent so we are OK if we close it twice but 
please double check. The 2.2 patch then merges without conflicts into 3.0.

CI should eventually appear here:

http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-2.1-dtest
http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-2.1-testall

http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-2.2-dtest
http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-2.2-testall

http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-3.0-dtest
http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-10534-3.0-testall


> 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
>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.(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 
> 

[jira] [Commented] (CASSANDRA-10534) CompressionInfo not being fsynced on close

2015-10-16 Thread Sharvanath Pathak (JIRA)

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

Sharvanath Pathak commented on CASSANDRA-10534:
---

@benedict @stefania thanks for taking quick action on it.

> 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
>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.(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.(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 
>