You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Benedict (JIRA)" <ji...@apache.org> on 2017/08/17 08:27:00 UTC

[jira] [Comment Edited] (CASSANDRA-13747) Fix short read protection

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

Benedict edited comment on CASSANDRA-13747 at 8/17/17 8:26 AM:
---------------------------------------------------------------

The patch looks good, but while reviewing I got a little suspicious of the modified line {{DataResolver:479}}, as it seemed that {{n}} and {{x}} were the wrong way around... and, reading the comment of intent directly above, and reproducing the calculation, they are indeed.

-Assuming, now correctly defined, that {{n <= x}}, this also obviates the need for the {{Math.max(x, 1)}} you have introduced.  This must be true, given that we can only have a short-read triggered in the case that we have yielded too few rows, so we must have fewer than we requested (even if other rows we didn't know about were introduced by other peers).-

This is _probably_ a significant enough bug that it warrants its own ticket for record keeping, though I'm fairly agnostic on that decision.  

I'm a little concerned about our current short read behaviour, as right now it seems we should be requesting exactly one row, for any size of under-read, which could mean extremely poor performance in case of large under-reads.

I would suggest that the outer unconditional {{Math.max}} is a bad idea, has been (poorly) insulating us from this error, and that we should first be asserting that the calculation yields a value {{>= 0}} before setting to 1.


was (Author: benedict):
The patch looks good, but while reviewing I got a little suspicious of the modified line {{DataResolver:479}}, as it seemed that {{n}} and {{x}} were the wrong way around... and, reading the comment of intent directly above, and reproducing the calculation, they are indeed.

Assuming, now correctly defined, that {{n <= x}}, this also obviates the need for the {{Math.max(x, 1)}} you have introduced.  This must be true, given that we can only have a short-read triggered in the case that we have yielded too few rows, so we must have fewer than we requested (even if other rows we didn't know about were introduced by other peers).

This is _probably_ a significant enough bug that it warrants its own ticket for record keeping, though I'm fairly agnostic on that decision.  

I'm a little concerned about our current short read behaviour, as right now it seems we should be requesting exactly one row, for any size of under-read, which could mean extremely poor performance in case of large under-reads.

I would suggest that the outer unconditional {{Math.max}} is a bad idea, has been (poorly) insulating us from this error, and that we should first be asserting that the calculation yields a value {{>= 0}} before setting to 1.

> Fix short read protection
> -------------------------
>
>                 Key: CASSANDRA-13747
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13747
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Coordination
>            Reporter: Aleksey Yeschenko
>            Assignee: Aleksey Yeschenko
>             Fix For: 3.0.x, 3.11.x
>
>
> {{ShortReadRowProtection.moreContents()}} expects that by the time we get to that method, the global post-reconciliation counter was already applied to the current partition. However, sometimes it won’t get applied, and the global counter continues counting with {{rowInCurrentPartition}} value not reset from previous partition, which in the most obvious case would trigger the assertion we are observing - {{assert !postReconciliationCounter.isDoneForPartition();}}. In other cases it’s possible because of this lack of reset to query a node for too few extra rows, causing unnecessary SRP data requests.
> Why is the counter not always applied to the current partition?
> The merged {{PartitionIterator}} returned from {{DataResolver.resolve()}} has two transformations applied to it, in the following order:
> {{Filter}} - to purge non-live data from partitions, and to discard empty partitions altogether (except for Thrift)
> {{Counter}}, to count and stop iteration
> Problem is, {{Filter}} ’s {{applyToPartition()}} code that discards empty partitions ({{closeIfEmpty()}} method) would sometimes consume the iterator, triggering short read protection *before* {{Counter}} ’s {{applyToPartition()}} gets called and resets its {{rowInCurrentPartition}} sub-counter.
> We should not be consuming iterators until all transformations are applied to them. For transformations it means that they cannot consume iterators unless they are the last transformation on the stack.
> The linked branch fixes the problem by splitting {{Filter}} into two transformations. The original - {{Filter}} - that does filtering within partitions - and a separate {{EmptyPartitionsDiscarder}}, that discards empty partitions from {{PartitionIterators}}. Thus {{DataResolve.resolve()}}, when constructing its {{PartitionIterator}}, now does merge first, then applies {{Filter}}, then {{Counter}}, and only then, as its last (third) transformation - the {{EmptyPartitionsDiscarder}}. Being the last one applied, it’s legal for it to consume the iterator, and triggering {{moreContents()}} is now no longer a problem.
> Fixes: [3.0|https://github.com/iamaleksey/cassandra/commits/13747-3.0], [3.11|https://github.com/iamaleksey/cassandra/commits/13747-3.11], [4.0|https://github.com/iamaleksey/cassandra/commits/13747-4.0]. dtest [here|https://github.com/iamaleksey/cassandra-dtest/commits/13747].



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org