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

Avoid trying to keep track of RTs for endpoints we won't write to during read repair

    XMLWordPrintableJSON

    Details

      Description

      During range movements we can have responses for endpoints we won't write to, in this case we don't collect {{PartitionUpdate}}s for those endpoints, but we do try to update the range tombstones. This causes us to fail with this assertion error since we assume a partition deletion to be LIVE if there is no current partition update getting built for an endpoint.

      Caused by: java.lang.AssertionError: Error merging RTs on distributed_test_keyspace.tbl: merged=null, versions=[null, Marker INCL_START_BOUND(20)@80/1598882605, Marker INCL_START_BOUND(20)@80/1598882605], sources={[Full(/127.0.0.3:7012,(-3,4611686018427387899]), Full(/127.0.0.1:7012,(-3,4611686018427387899]), Full(/127.0.0.2:7012,(-3,4611686018427387899])]}, debug info:
       /127.0.0.3:7012 => [distributed_test_keyspace.tbl] key=test1 partition_deletion=deletedAt=100, localDeletion=1598882605 columns=[[] | []] repaired_digest= repaired_digest_conclusive==true,
      /127.0.0.1:7012 => [distributed_test_keyspace.tbl] key=test1 partition_deletion=deletedAt=50, localDeletion=1598882605 columns=[[] | []] repaired_digest= repaired_digest_conclusive==true
          Marker INCL_START_BOUND(20)@80/1598882605
          Marker INCL_END_BOUND(40)@80/1598882605,
      /127.0.0.2:7012 => [distributed_test_keyspace.tbl] key=test1 partition_deletion=deletedAt=50, localDeletion=1598882605 columns=[[] | []] repaired_digest= repaired_digest_conclusive==true
          Marker INCL_START_BOUND(20)@80/1598882605
          Marker INCL_END_BOUND(40)@80/1598882605
      	at org.apache.cassandra.service.reads.DataResolver$2$1.onMergedRangeTombstoneMarkers(DataResolver.java:384)
      	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:592)
      	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:541)
      	at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:219)
      	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:523)
      	at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:391)
      	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.FilteredRows.isEmpty(FilteredRows.java:50)
      	at org.apache.cassandra.db.transform.EmptyPartitionsDiscarder.applyToPartition(EmptyPartitionsDiscarder.java:27)
      	at org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:97)
      	at org.apache.cassandra.service.StorageProxy$6.hasNext(StorageProxy.java:1768)
      	at org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:777)
      	at org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:425)
      	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:296)
      	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:246)
      	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:88)
      	at org.apache.cassandra.distributed.impl.Coordinator.executeInternal(Coordinator.java:100)
      	at org.apache.cassandra.distributed.impl.Coordinator.lambda$executeWithResult$0(Coordinator.java:62)
      	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 io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
      	at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.AssertionError: currentDeletion=deletedAt=100, localDeletion=1598882605, marker=Marker INCL_START_BOUND(20)@80/1598882605
      	at org.apache.cassandra.service.reads.repair.RowIteratorMergeListener.onMergedRangeTombstoneMarkers(RowIteratorMergeListener.java:296)
      	at org.apache.cassandra.service.reads.DataResolver$2$1.onMergedRangeTombstoneMarkers(DataResolver.java:370)
      	... 25 more
      

        Attachments

          Activity

            People

            • Assignee:
              marcuse Marcus Eriksson
              Reporter:
              marcuse Marcus Eriksson
              Authors:
              Marcus Eriksson
              Reviewers:
              Benedict Elliott Smith
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved: