cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefan Podkowinski (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-11427) Range slice queries CL > ONE trigger read-repair of purgeable tombstones
Date Wed, 30 Mar 2016 12:30:25 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-11427?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Stefan Podkowinski updated CASSANDRA-11427:
-------------------------------------------
    Summary: Range slice queries CL > ONE trigger read-repair of purgeable tombstones 
(was: Range slice queries CL > ONE trigger read-repair of purgable tombstones)

> 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)

Mime
View raw message