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

Stefania commented on CASSANDRA-12582:
--------------------------------------

It's worth noting that the exception I reproduced is actually different that 
the one in the ticket description. It seems the exception in the description is 
failing to read a static row whilst the exception I reproduced is failing to 
read a regular row (a clustering to be exact), I hope the workaround in the 
serialization header covers both, but I cannot reproduce the exact exception in 
the description. 

{code}
java.lang.RuntimeException: 
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/stefi/git/cstar/cassandra/data/data/issue12582/apples_by_tree-3980ba90809c11e6b62e6be1d44ebd9b/mc-1-big-Data.db
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2470)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_101]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
[main/:na]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/stefi/git/cstar/cassandra/data/data/issue12582/apples_by_tree-3980ba90809c11e6b62e6be1d44ebd9b/mc-1-big-Data.db
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:353)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:219)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.UnfilteredRows.isEmpty(UnfilteredRows.java:58)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:65)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:24)
 ~[main/:na]
        at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:295)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:145)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:138)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:134)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76) 
~[main/:na]
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:320) 
~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1796)
 ~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2466)
 ~[main/:na]
        ... 5 common frames omitted
Caused by: java.io.IOException: Corrupt flags value for clustering prefix 
(isStatic flag set): 160
        at 
org.apache.cassandra.db.ClusteringPrefix$Deserializer.prepare(ClusteringPrefix.java:422)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.prepareNext(UnfilteredDeserializer.java:172)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.hasNext(UnfilteredDeserializer.java:153)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:126)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:153)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:340)
 ~[main/:na]
        ... 26 common frames omitted
{code}

> Removing static column results in ReadFailure due to CorruptSSTableException
> ----------------------------------------------------------------------------
>
>                 Key: CASSANDRA-12582
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12582
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local Write-Read Paths
>         Environment: Cassandra 3.0.8
>            Reporter: Evan Prothro
>            Assignee: Stefania
>            Priority: Critical
>              Labels: compaction, corruption, drop, read, static
>             Fix For: 3.0.x, 3.x
>
>         Attachments: 12582.cdl, 12582_reproduce.sh
>
>
> We ran into an issue on production where reads began to fail for certain 
> queries, depending on the range within the relation for those queries. 
> Cassandra system log showed an unhandled {{CorruptSSTableException}} 
> exception.
> CQL read failure:
> {code}
> ReadFailure: code=1300 [Replica(s) failed to execute read] message="Operation 
> failed - received 0 responses and 1 failures" info={'failures': 1, 
> 'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'}
> {code}
> Cassandra exception:
> {code}
> WARN  [SharedPool-Worker-2] 2016-08-31 12:49:27,979 
> AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread 
> Thread[SharedPool-Worker-2,5,main]: {}
> java.lang.RuntimeException: 
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> /usr/local/apache-cassandra-3.0.8/data/data/issue309/apples_by_tree-006748a06fa311e6a7f8ef8b642e977b/mb-1-big-Data.db
>   at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2453)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
>  [apache-cassandra-3.0.8.jar:3.0.8]
>   at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.8.jar:3.0.8]
>   at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: 
> Corrupted: 
> /usr/local/apache-cassandra-3.0.8/data/data/issue309/apples_by_tree-006748a06fa311e6a7f8ef8b642e977b/mb-1-big-Data.db
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:343)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.isReverseOrder(LazilyInitializedUnfilteredRowIterator.java:65)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.isReverseOrder(LazilyInitializedUnfilteredRowIterator.java:66)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:62)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:24)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:295)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:127)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:289) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1796)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2449)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   ... 5 common frames omitted
> Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: 
> Corrupted: 
> /usr/local/apache-cassandra-3.0.8/data/data/issue309/apples_by_tree-006748a06fa311e6a7f8ef8b642e977b/mb-1-big-Data.db
>   at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:130)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.columniterator.SSTableIterator.<init>(SSTableIterator.java:46)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:69)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:338)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   ... 19 common frames omitted
> Caused by: java.io.IOException: Corrupt (negative) value length encountered
>   at 
> org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:399) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.BufferCell$Serializer.deserialize(BufferCell.java:302)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.readSimpleColumn(UnfilteredSerializer.java:462)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:440)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeStaticRow(UnfilteredSerializer.java:381)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.readStaticRow(AbstractSSTableIterator.java:179)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.<init>(AbstractSSTableIterator.java:103)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>   ... 22 common frames omitted
> {code}
> After debugging, it appears that a previously dropped static column (weeks 
> prior) was the instigator of the issue. As a workaround we added back the 
> column, restarted all cassandra processes within the cluster, and the read 
> error and corruption exception went away.
> Attached is a script to reproduce with a simple schema.
> Also noteworthy (and shown in the script) is that when in this state, 
> compaction silently failed (exit 0) to remove the dropped static columns from 
> the "corrupted" sstable.



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

Reply via email to