Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-13337

Dropping column results in "corrupt" SSTable

    XMLWordPrintableJSON

Details

    • Normal

    Description

      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

      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)
      
      

      Attachments

        Activity

          People

            slebresne Sylvain Lebresne
            jborgstrom Jonas Borgström
            Sylvain Lebresne
            Alex Petrov
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: