[ https://issues.apache.org/jira/browse/CASSANDRA-11427?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15226316#comment-15226316 ]
Sylvain Lebresne commented on CASSANDRA-11427: ---------------------------------------------- The patch works but it has 2 small inefficiencies: # it purges on the coordinator, but there is no point for purgeable tombstones to leave the replicas in the first place. # single partition queries already do that filtering on replicas (in {{ColumnFamilyStore.getColumnFamily(QueryFilter)}}), and since {{scheduleRepairs()}} is shared by single partition and range queries, that adds unecessary work for single partition ones. Overall, I think it would be better (and actually slightly easier) to just call {{removeDeletedCF}} in {{ColumnFamilyStore.filter(AbstractScanIterator, ExtendedFilter)}}. But I would also note that: # 3.0 is not affected # this only concerns partition and range tombstones, so this is imo a fairly minor efficiency problem, not a correction issue. So in particular, this doesn't met the "critical issues only" bar for 2.1. So that I wonder if it's worth taking any risk just for 2.2. Not too opposed to it though if someone cares deeply. > Range slice queries CL > ONE trigger read-repair of purgeable tombstones > ------------------------------------------------------------------------ > > Key: CASSANDRA-11427 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11427 > Project: Cassandra > Issue Type: Bug > Reporter: Stefan Podkowinski > Assignee: Stefan Podkowinski > Fix For: 2.1.x, 2.2.x > > Attachments: 11427-2.1.patch > > > Range queries will trigger read repairs for purgeable tombstones on hosts > that already compacted given tombstones. Clusters with periodical jobs for > scanning data ranges will likely see tombstones ressurected through RRs just > to have them compacted again later at the destination host. > Executing range queries (e.g. for reading token ranges) will compare the > actual data instead of using digests when executed with CL > ONE. Responses > will be consolidated by {{RangeSliceResponseResolver.Reducer}}, where the > result of {{RowDataResolver.resolveSuperset}} is used as the reference > version for the results. {{RowDataResolver.scheduleRepairs}} will then send > the superset to all nodes that returned a different result before. > Unfortunately this does also involve cases where the superset is just made up > of purgeable tombstone(s) that already have been compacted on the other > nodes. In this case a read-repair will be triggered for transfering the > purgeable tombstones to all other nodes nodes that returned an empty result. > The issue can be reproduced with the provided dtest or manually using the > following steps: > {noformat} > create keyspace test1 with replication = { 'class' : 'SimpleStrategy', > 'replication_factor' : 2 }; > use test1; > create table test1 ( a text, b text, primary key(a, b) ) WITH compaction = > {'class': 'SizeTieredCompactionStrategy', 'enabled': 'false'} AND > dclocal_read_repair_chance = 0 AND gc_grace_seconds = 0; > delete from test1 where a = 'a'; > {noformat} > {noformat} > ccm flush; > ccm node2 compact; > {noformat} > {noformat} > use test1; > consistency all; > tracing on; > select * from test1; > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)