You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2017/03/20 15:25:41 UTC

[jira] [Updated] (CASSANDRA-13337) Dropping column results in "corrupt" SSTable

     [ https://issues.apache.org/jira/browse/CASSANDRA-13337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne updated CASSANDRA-13337:
-----------------------------------------
    Fix Version/s: 3.11.x
                   3.0.x
    Reproduced In: 3.10, 3.0.12  (was: 3.0.12, 3.10)
           Status: Patch Available  (was: Open)

This is definitively wrong, attaching patch for fix (including an unit test to reproduce).
| [13337-3.0|https://github.com/pcmanus/cassandra/commits/13337-3.0] | [utests|http://cassci.datastax.com/job/pcmanus-13337-3.0-testall] | [dtests|http://cassci.datastax.com/job/pcmanus-13337-3.0-dtest] |
| [13337-3.11|https://github.com/pcmanus/cassandra/commits/13337-3.11] | [utests|http://cassci.datastax.com/job/pcmanus-13337-3.11-testall] | [dtests|http://cassci.datastax.com/job/pcmanus-13337-3.11-dtest] |

I'll note that the patch basically disable the error message seen here, instead simply ignoring empty rows from disk since they can happen. I suppose it would be possible to do a more involved checking to make sure we didn't wrote something actually empty, but I'm not sure at all it's worth the trouble (not the cost of that check on a pretty hot path) especially given that expecting non-empty rows was wrong no only due to dropping, but also because we can actually skip columns due to the {{ColumnFilter}} within {{SerializationHelper}}. I believe the latter would only potentially impact thrift queries, which may be why nobody as yet reported that problem, but it's still wrong.


> Dropping column results in "corrupt" SSTable
> --------------------------------------------
>
>                 Key: CASSANDRA-13337
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13337
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>            Reporter: Jonas Borgström
>            Assignee: Sylvain Lebresne
>             Fix For: 3.0.x, 3.11.x
>
>
> It seems like dropping a column can make SSTables containing rows with writes to only the dropped column will become uncompactable.
> Also Cassandra <= 3.9 and <= 3.0.11 will even refuse to start with the same stack trace
> {code}
> cqlsh -e "create keyspace test with replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"
> cqlsh -e "create table test.test(pk text primary key, x text, y text)"
> cqlsh -e "update test.test set x='1' where pk='1'"
> nodetool flush
> cqlsh -e "update test.test set x='1', y='1' where pk='1'"
> nodetool flush
> cqlsh -e "alter table test.test drop x"
> nodetool compact test test
> error: Corrupt empty row found in unfiltered partition
> -- StackTrace --
> java.io.IOException: Corrupt empty row found in unfiltered partition
> 	at org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:382)
> 	at org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:87)
> 	at org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:65)
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> 	at org.apache.cassandra.io.sstable.SSTableIdentityIterator.doCompute(SSTableIdentityIterator.java:123)
> 	at org.apache.cassandra.io.sstable.SSTableIdentityIterator.computeNext(SSTableIdentityIterator.java:100)
> 	at org.apache.cassandra.io.sstable.SSTableIdentityIterator.computeNext(SSTableIdentityIterator.java:30)
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> 	at org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
> 	at org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> 	at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369)
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189)
> 	at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158)
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:509)
> 	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:369)
> 	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> 	at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129)
> 	at org.apache.cassandra.db.transform.UnfilteredRows.isEmpty(UnfilteredRows.java:58)
> 	at org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:67)
> 	at org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:26)
> 	at org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)
> 	at org.apache.cassandra.db.compaction.CompactionIterator.hasNext(CompactionIterator.java:227)
> 	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:190)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> 	at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:89)
> 	at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
> 	at org.apache.cassandra.db.compaction.CompactionManager$8.runMayThrow(CompactionManager.java:610)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> 	at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)