cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Peter Schuller (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-2643) read repair/reconciliation breaks slice based iteration at QUORUM
Date Thu, 12 May 2011 19:05:47 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-2643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13032589#comment-13032589
] 

Peter Schuller commented on CASSANDRA-2643:
-------------------------------------------

Unless I'm off, the problem with deleted columns is fairly similar to that of deleted rows.
Like with range ghosts, an option would be to propagate tombstones to clients.

Another option is to make read responses include the range for which they are authoritative,
and then only consider the intersection of all responses' authoritative ranges when reconciling
results. The authoritative range of the response would have to be communicated back to the
client, such that it can continue iterating from the appropriate column name even without
actually receiving a column for that name.

Other than failing requests with a new kind of error, I don't see a good way to fix the tombstone
case (the over-shoot case can be fixed by just capping results) without changing the protocol.
For obvious reason's we don't want the co-originating node to go into a potentially unbounded
re-try spin until sufficient results are obtained from all nodes participating.

FWIW, returning iteration state feels pretty clean to me (it's what our layer implements on
top to enable easier iteration). It is also compatible with future improvements to e.g. cap
the size of a response by bytes for safely slicing over columns whose size you do not know.
By removing the strict requirement to deliver exactly the number of asked-for columns else
it be interpreted as "out of columns", significant flexibility is gained. So if the only option
for a clean fix is truly to change the protocol, at least other positive benefits may be had.

> read repair/reconciliation breaks slice based iteration at QUORUM
> -----------------------------------------------------------------
>
>                 Key: CASSANDRA-2643
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2643
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7.5
>            Reporter: Peter Schuller
>            Priority: Critical
>         Attachments: slicetest.py
>
>
> In short, I believe iterating over columns is impossible to do reliably with QUORUM due
to the way reconciliation works.
> The problem is that the SliceQueryFilter is executing locally when reading on a node,
but no attempts seem to be made to consider limits when doing reconciliation and/or read-repair
(RowRepairResolver.resolveSuperset() and ColumnFamily.resolve()).
> If a node slices and comes up with 100 columns, and another node slices and comes up
with 100 columns, some of which are unique to each side, reconciliation results in > 100
columns in the result set. In this case the effect is limited to "client gets more than asked
for", but the columns still accurately represent the range. This is easily triggered by my
test-case.
> In addition to the client receiving "too many" columns, I believe some of them will not
be satisfying the QUORUM consistency level for the same reasons as with deletions (see discussion
below).
> Now, there *should* be a problem for tombstones as well, but it's more subtle. Suppose
A has:
>   1
>   2
>   3
>   4
>   5
>   6
> and B has:
>   1
>   del 2
>   del 3
>   del 4
>   5
>   6 
> If you now slice 1-6 with count=3 the tombstones from B will reconcile with those from
A - fine. So you end up getting 1,5,6 back. This made it a bit difficult to trigger in a test
case until I realized what was going on. At first I was "hoping" to see a "short" iteration
result, which would mean that the process of iterating until you get a short result will cause
spurious "end of columns" and thus make it impossible to iterate correctly.
> So; due to 5-6 existing (and if they didn't, you legitimately reached end-of-columns)
we do indeed get a result of size 3 which contains 1,5 and 6. However, only node B would have
contributed columns 5 and 6; so there is actually no QUORUM consistency on the co-ordinating
node with respect to these columns. If node A and C also had 5 and 6, they would not have
been considered.
> Am I wrong?
> In any case; using script I'm about to attach, you can trigger the over-delivery case
very easily:
> (0) disable hinted hand-off to avoid that interacting with the test
> (1) start three nodes
> (2) create ks 'test' with rf=3 and cf 'slicetest'
> (3) ./slicetest.py hostname_of_node_C insert # let it run for a few seconds, then ctrl-c
> (4) stop node A
> (5) ./slicetest.py hostname_of_node_C insert # let it run for a few seconds, then ctrl-c
> (6) start node A, wait for B and C to consider it up
> (7) ./slicetest.py hostname_of_node_A slice # make A co-ordinator though it doesn't necessarily
matter
> You can also pass 'delete' (random deletion of 50% of contents) or 'deleterange' (delete
all in [0.2,0.8]) to slicetest, but you don't trigger a short read by doing that (see discussion
above).

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message