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

upgradesstables fails after upgrading from 2.1.x to 3.0.11

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • 3.0.13, 3.11.0
    • None
    • None
    • Normal

    Description

      I tried to execute nodetool upgradesstables after upgrading cluster from 2.1.16 to 3.0.11, but it fails when upgrading a table with 2i.

      This problem can be reproduced as follows.

      $ ccm create test -v 2.1.16 -n 1 -s
      $ ccm node1 cqlsh  -e "CREATE KEYSPACE test WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}"
      $ ccm node1 cqlsh  -e "CREATE TABLE test.test(k1 text, k2 text, PRIMARY KEY( k1 ));"
      $ ccm node1 cqlsh  -e "CREATE INDEX k2 ON test.test(k2);"
       
      $ ccm node1 cqlsh  -e "INSERT INTO test.test (k1, k2 ) VALUES ( 'a', 'a') ;"
      $ ccm node1 cqlsh  -e "INSERT INTO test.test (k1, k2 ) VALUES ( 'a', 'b') ;"
       
      $ ccm node1 nodetool flush
       
      $ for i in `seq 1 `; do ccm node${i} stop; ccm node${i} setdir -v3.0.11;ccm node${i} start; done
      $ ccm node1 nodetool upgradesstables test test
      Traceback (most recent call last):
        File "/home/y/bin/ccm", line 86, in <module>
          cmd.run()
        File "/home/y/lib/python2.7/site-packages/ccmlib/cmds/node_cmds.py", line 267, in run
          stdout, stderr = self.node.nodetool(" ".join(self.args[1:]))
        File "/home/y/lib/python2.7/site-packages/ccmlib/node.py", line 742, in nodetool
          raise NodetoolError(" ".join(args), exit_status, stdout, stderr)
      ccmlib.node.NodetoolError: Nodetool command '/home/zzheng/.ccm/repository/3.0.11/bin/nodetool -h localhost -p 7100 upgradesstables test test' failed; exit status: 2; stderr: WARN  06:29:08 Only 10476 MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots
      error: null
      -- StackTrace --
      java.lang.AssertionError
      	at org.apache.cassandra.db.rows.Rows.collectStats(Rows.java:70)
      	at org.apache.cassandra.io.sstable.format.big.BigTableWriter$StatsCollector.applyToRow(BigTableWriter.java:197)
      	at org.apache.cassandra.db.transform.BaseRows.applyOne(BaseRows.java:116)
      	at org.apache.cassandra.db.transform.BaseRows.add(BaseRows.java:107)
      	at org.apache.cassandra.db.transform.UnfilteredRows.add(UnfilteredRows.java:41)
      	at org.apache.cassandra.db.transform.Transformation.add(Transformation.java:156)
      	at org.apache.cassandra.db.transform.Transformation.apply(Transformation.java:122)
      	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:147)
      	at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:125)
      	at org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter.realAppend(DefaultCompactionWriter.java:57)
      	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:109)
      	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:195)
      	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$5.execute(CompactionManager.java:415)
      	at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:307)
      	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)
      

      The result of dumping the 2i sstable is as follows.

      [
      {"key": "a",
       "cells": [["61",1488961273,1488961269822817,"d"]]},
      {"key": "b",
       "cells": [["61","",1488961273015759]]}
      ]
      

      This problem is occurred by the tombstone row. When this row is processed in LegacyLayout.java, it will be treated as a row maker.
      https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/db/LegacyLayout.java#L1195
      Then the deletion info will be lost.

      As a result, the row will be a empty row, which causes the assertion error.

      To avoid this, I added the code to add row deletion info when the row is a tombstone and not a row marker, and it works as I expect, which means that upgradesstables succeeds and row deletion info is remained.

      However I don't understand whether this change will cause another problem. Anyway, I submit my patch as a reference.

      Attachments

        1. 13320.patch
          1.0 kB
          Zhongxiang Zheng

        Issue Links

          Activity

            People

              samt Sam Tunnicliffe
              zzheng Zhongxiang Zheng
              Sam Tunnicliffe
              Benjamin Lerer
              Votes:
              1 Vote for this issue
              Watchers:
              9 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: