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

AssertionError while trying to upgrade 2.2.11 -> 3.11.1

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • 3.11.14
    • Legacy/Core
    • Tables have been created in 2.2.11 using thrift and have supercolumns

    Description

      We're trying to upgrade a test cluster from Cassandra 2.2.11 to Cassandra 3.11.1. The tables have been created using thrift and have supercolumns. When I try to run nodetool upgradesstables I get the following:

      error: null
      -- StackTrace --
      java.lang.AssertionError
      	at org.apache.cassandra.db.rows.BufferCell.<init>(BufferCell.java:42)
      	at org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1242)
      	at org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1185)
      	at org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:498)
      	at org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:472)
      	at org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:306)
      	at org.apache.cassandra.io.sstable.SSTableSimpleIterator$OldFormatIterator.computeNext(SSTableSimpleIterator.java:188)
      	at org.apache.cassandra.io.sstable.SSTableSimpleIterator$OldFormatIterator.computeNext(SSTableSimpleIterator.java:140)
      	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
      	at org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:122)
      	at org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:100)
      	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$TrivialOneToOne.computeNext(MergeIterator.java:484)
      	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
      	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:499)
      	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:359)
      	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
      	at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
      	at org.apache.cassandra.db.transform.UnfilteredRows.isEmpty(UnfilteredRows.java:74)
      	at org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:75)
      	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:233)
      	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:196)
      	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      	at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:85)
      	at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
      	at org.apache.cassandra.db.compaction.CompactionManager$5.execute(CompactionManager.java:428)
      	at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:315)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
      	at java.lang.Thread.run(Thread.java:748)
      

      We also tried to upgrade to 3.0.15 instead and had a different error:

      ERROR 11:00:40 Exception in thread Thread[CompactionExecutor:1,1,main]
      java.lang.IllegalStateException: [ColumnDefinition{name=key, type=org.apache.cassandra.db.marshal.BytesType, kind=PARTITION_KEY, position=0}, ColumnDefinition{name=, type=org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.BytesType,org.apache.cassandra.db.marshal.BytesType), kind=REGULAR, position=-1}] is not a subset of []
          at org.apache.cassandra.db.Columns$Serializer.encodeBitmap(Columns.java:532) ~[main/:na]
          at org.apache.cassandra.db.Columns$Serializer.serializedSubsetSize(Columns.java:484) ~[main/:na]
          at org.apache.cassandra.db.rows.UnfilteredSerializer.serializedRowBodySize(UnfilteredSerializer.java:290) ~[main/:na]
          at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:169) ~[main/:na]
          at org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:114) ~[main/:na]
          at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:144) ~[main/:na]
          at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:112) ~[main/:na]
          at org.apache.cassandra.db.ColumnIndex.writeAndBuildIndex(ColumnIndex.java:52) ~[main/:na]
          at org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:149) ~[main/:na]
          at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:125) ~[main/:na]
          at org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter.realAppend(MaxSSTableSizeWriter.java:88) ~[main/:na]
          at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:109) ~[main/:na]
          at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:195) ~[main/:na]
          at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[main/:na]
          at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:89) ~[main/:na]
          at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61) ~[main/:na]
          at org.apache.cassandra.db.compaction.CompactionManager$5.execute(CompactionManager.java:424) ~[main/:na]
          at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:311) ~[main/:na]
          at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_151]
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_151]
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_151]
          at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) [main/:na]
          at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_151]
      

      Attached are a set of sstables that reproduce the issue.

      Attachments

        1. 14113-3.0.txt
          6 kB
          Marten Kenbeek
        2. data.tar.gz
          66 kB
          Guillaume Herail

        Activity

          People

            knbk Marten Kenbeek
            xiu Guillaume Herail
            Marten Kenbeek
            Benjamin Lerer
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: