[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


Short term (until you can upgrade), you probably want to avoid calling the 
{{getDroppableTombstoneRatio()}} mbean.

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

I'm happy as long as it's fixed! Thanks for the support here! :)

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


[~hkroger] thanks so much for the detailed report and your patience while we 
fix this right. Do you mind if I close this as a dupe of 13756, the new patches 
there should fix it (and ultimately, it's the same issue - it wasn't thread 
safe)?


> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


I'd rather just change it to a builder in 3.0 and 3.11 for safety in case 
someone else comes along and tries to misuse it again in the future?

I've got the 3.0 version done, working on 3.11 now. Would you do a second 
review with Jason just to be safe? 





> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


Let's just call this a dupe of 13756, and fix it all there.

I'm pushing changes to 3.0 and 3.11 that transform this into a builder, but 
also changes to CANONICAL for safety. If you have time, can you do a second 
review alongside Jason so we can squash this once and be confident it's fixed?

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13752:
-

bq. we should be using a "builder" for StreamingHistogram
looks like we already do this in trunk

so lets just change to CANONICAL and do CASSANDRA-13756 - wdyt [~jjirsa]?

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-29 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13752:
-

So this is because we reuse the same StreamingHistogram when we [open sstables 
early|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java#L292]
 - the early opened sstable will be using the streaming histogram that 
compaction is still building. Since CASSANDRA-13038 we can modify the contents 
of the StreamingHistogram when we call {{sum()}} ({{spool}} might be compacted 
into {{bin}}). So, if someone calls the 
{{ColumnFamilyStoreMBean#getDroppableTombstoneRatio}} at the wrong time we 
could get either the CME from CASSANDRA-13756 or this corruption.

Making StreamingHistogram thread safe is one way of fixing this, but I would 
argue that we should be using a "builder" for StreamingHistogram - we should 
never access the SH while building it and for early opened sstables we should 
call {{.build()}} on the StreamingHistogramBuilder and get a copy of the 
internal state.

Also, we should not query LIVE sstables 
[here|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L2589]
 - it should be using {{SSTableSet.CANONICAL}} (this is probably enough to fix 
this for now - this is the only way I can see that we access the 
sstablemetadata in early opened sstables).

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 

[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-28 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

[~jjirsa] We saw off by one in both directions. I will check with the guy who 
wrote a tool to fix the stats if he saw other values.

Also, might be that this doesn't fix the original problem causing it but at 
least it shouldn't write size different from amount of entries actually 
written. :P

Anyways, if there is some better approach, by all means.

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-28 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

[~krummas] We are not sure if it was compaction or streaming. But the sizes are 
8-45GB and the typical size was around 20-25GB.

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-28 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


[~hkroger] - I thought I remember reading (perhaps in IRC) that you opened this 
in a debugger and saw the corruption was that the length of the bins was off by 
one. Was the length field 1 less than the bins, or 1 greater than actual number 
of bins? 


> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-28 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13752:
-

[~hkroger] how was the corrupted sstable written? 
(compaction/flush/anticompaction?)

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-28 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


[~jasobrown] and I have been chatting about this today. We're looking for a 
solution. I don't think your patch actually solves this problem, and it may 
have existed before 13038 (though 13038 may have made it more likely).


> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
> Fix For: 3.11.1
>
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-25 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

[~jjirsa] Could downgrade to 3.10 help as a quick fix?

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-23 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


I'll take review in conjunction with CASSANDRA-13756


> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Assignee: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-23 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

[~whangsf] I ported the fix to 3.0 as well (assuming the same race condition 
can happen there as well).

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-23 Thread Andrew Whang (JIRA)

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

Andrew Whang commented on CASSANDRA-13752:
--

Can we get this patched to 3.0 as well?

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-22 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

I created a branch with potential fix for this particular problem

||Branch||utest||dtest||
|[3.11|https://circleci.com/gh/hkroger/cassandra/tree/cassandra-3.11-13752]|[3.11
 
circle|https://circleci.com/gh/hkroger/cassandra/tree/cassandra-3.11-13752]|???|


> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-21 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

I think I got it confirmed. StreamingHistogram size stored had actually value 
that was less than the amount of elements serialized. It was off by one.

Only thing I am worried is that if CASSANDRA-13756 patch fixes also the 
serialization issue.

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-21 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

I debugged the code by loading Cassandra in a debugger and opening that sstable 
(with empty data & index) and after deserializing StreamingHistogram the data 
seems to be garbage.

Could this be a side effect of this bug: 
https://issues.apache.org/jira/browse/CASSANDRA-13756 ?

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-10 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

Background information:
- Incremental repairs are being run regularly
- Same cluster suffers also from this: 
https://issues.apache.org/jira/browse/CASSANDRA-13718
- To mitigate the previous bug we have run full repairs on the full cluster on 
problematic tables
- Lucene index plugin is installed but not in use in the keyspace in question
- Cassandra version was 2.2.8 but was upgraded to 3.11.0
- 4 nodes in DC1 (DC2 not connected atm.), RF=3
- Upgrade to 3.11 was done maybe 1,5 weeks ago
- Cluster has been running since may '17

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-10 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-13752:


Any additional context you can provide - how old is the cluster? Have you 
changed anything recently? When did you upgrade to 3.11.0? How long before you 
saw those errors? Do you run repairs? Incremental repairs or full? Anything 
else in the logs that looks atypical? 

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13752) Corrupted SSTables created in 3.11

2017-08-10 Thread JIRA

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

Hannu Kröger commented on CASSANDRA-13752:
--

This has happened on 2 servers for total of at least 3 sstables so far and I 
can read those files with unix tools like cat so it doesn't seem like it's a FS 
or HW issue.

> Corrupted SSTables created in 3.11
> --
>
> Key: CASSANDRA-13752
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13752
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Hannu Kröger
>Priority: Blocker
>
> We have discovered issues with corrupted SSTables. 
> {code}
> ERROR [SSTableBatchOpen:22] 2017-08-03 20:19:53,195 SSTableReader.java:577 - 
> Cannot read sstable 
> /cassandra/data/mykeyspace/mytable-7a4992800d5611e7b782cb90016f2d17/mc-35556-big=[Data.db,
>  Statistics.db, Summary.db, Digest.crc32, CompressionInfo.db, TOC.txt, 
> Index.db, Filter.db]; other IO error, skipping table
> java.io.EOFException: EOF after 1898 bytes out of 21093
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:377)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:325)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.StatsMetadata$StatsMetadataSerializer.deserialize(StatsMetadata.java:231)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:122)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:93)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:488)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:396)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$5.run(SSTableReader.java:561)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_111]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_111]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_111]
> at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
> {code}
> Files look like this:
> {code}
> -rw-r--r--. 1 cassandra cassandra 3899251 Aug  7 08:37 
> mc-6166-big-CompressionInfo.db
> -rw-r--r--. 1 cassandra cassandra 16874421686 Aug  7 08:37 mc-6166-big-Data.db
> -rw-r--r--. 1 cassandra cassandra  10 Aug  7 08:37 
> mc-6166-big-Digest.crc32
> -rw-r--r--. 1 cassandra cassandra 2930904 Aug  7 08:37 
> mc-6166-big-Filter.db
> -rw-r--r--. 1 cassandra cassandra   75880 Aug  7 08:37 
> mc-6166-big-Index.db
> -rw-r--r--. 1 cassandra cassandra   13762 Aug  7 08:37 
> mc-6166-big-Statistics.db
> -rw-r--r--. 1 cassandra cassandra  882008 Aug  7 08:37 
> mc-6166-big-Summary.db
> -rw-r--r--. 1 cassandra cassandra  92 Aug  7 08:37 mc-6166-big-TOC.txt
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org