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

[jira] [Comment Edited] (CASSANDRA-11500) Obsolete MV entry may not be properly deleted

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

ZhaoYang edited comment on CASSANDRA-11500 at 8/1/17 6:27 AM:
--------------------------------------------------------------

WIP [branch|https://github.com/jasonstack/cassandra/commits/CASSANDRA-11500-cell] [dtest|https://github.com/riptano/cassandra-dtest/commits/CASSANDRA-11500]


Changed:
*  Extra "VirtualCell"(kind of special LivenessInfo for MV) in Row to maintain consistency of view row. It stores: 1. base column used in view PK and base column used in view filter conditions. if any of such column dead, entire view row dead, regardless LivenessInfo or DeletionTime status. or 2. unselected base columns. if any of such column alive, view's pk should be alive. 
* blocked dropping filter base column
* fix issue of creating view with token()  filter
* remove Row.Deletion, it was used to wrap DeletionTime with "Shadowable" flag
* fix missing partition deletion in view-update

Todo: 
more dtest


was (Author: jasonstack):
WIP [branch|https://github.com/jasonstack/cassandra/commits/CASSANDRA-11500-cell]

Changed:
*  Extra "VirtualCell"(kind of special LivenessInfo for MV) in Row to maintain liveness of view row. It stores: 1. base column used in view PK and base column used in view filter conditions. if any of such column dead, entire view row dead, regardless LivenessInfo or DeletionTime status. 2. unselected base columns. if any of such column alive, view's pk should be alive if it's not deleted by DeletionTime or those columns in <1>. 
* blocked dropping filter base column
* fix issue of creating view with token()  filter
* remove Row.Deletion, it was used to wrap DeletionTime with "Shadowable" flag

Todo: 
optimize in-memory/storage representation for "virtual-cells" to re-use AbstractRow/BTree
more dtest
optimize ViewUpdateGenerator process to reduce payload. eg, if view row has live pk or other live column, "virtualCells' unselected payload" is not necessary.

> Obsolete MV entry may not be properly deleted
> ---------------------------------------------
>
>                 Key: CASSANDRA-11500
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11500
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Materialized Views
>            Reporter: Sylvain Lebresne
>            Assignee: ZhaoYang
>
> When a Materialized View uses a non-PK base table column in its PK, if an update changes that column value, we add the new view entry and remove the old one. When doing that removal, the current code uses the same timestamp than for the liveness info of the new entry, which is the max timestamp for any columns participating to the view PK. This is not correct for the deletion as the old view entry could have other columns with higher timestamp which won't be deleted as can easily shown by the failing of the following test:
> {noformat}
> CREATE TABLE t (k int PRIMARY KEY, a int, b int);
> CREATE MATERIALIZED VIEW mv AS SELECT * FROM t WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (k, a);
> INSERT INTO t(k, a, b) VALUES (1, 1, 1) USING TIMESTAMP 0;
> UPDATE t USING TIMESTAMP 4 SET b = 2 WHERE k = 1;
> UPDATE t USING TIMESTAMP 2 SET a = 2 WHERE k = 1;
> SELECT * FROM mv WHERE k = 1; // This currently return 2 entries, the old (invalid) and the new one
> {noformat}
> So the correct timestamp to use for the deletion is the biggest timestamp in the old view entry (which we know since we read the pre-existing base row), and that is what CASSANDRA-11475 does (the test above thus doesn't fail on that branch).
> Unfortunately, even then we can still have problems if further updates requires us to overide the old entry. Consider the following case:
> {noformat}
> CREATE TABLE t (k int PRIMARY KEY, a int, b int);
> CREATE MATERIALIZED VIEW mv AS SELECT * FROM t WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (k, a);
> INSERT INTO t(k, a, b) VALUES (1, 1, 1) USING TIMESTAMP 0;
> UPDATE t USING TIMESTAMP 10 SET b = 2 WHERE k = 1;
> UPDATE t USING TIMESTAMP 2 SET a = 2 WHERE k = 1; // This will delete the entry for a=1 with timestamp 10
> UPDATE t USING TIMESTAMP 3 SET a = 1 WHERE k = 1; // This needs to re-insert an entry for a=1 but shouldn't be deleted by the prior deletion
> UPDATE t USING TIMESTAMP 4 SET a = 2 WHERE k = 1; // ... and we can play this game more than once
> UPDATE t USING TIMESTAMP 5 SET a = 1 WHERE k = 1;
> ...
> {noformat}
> In a way, this is saying that the "shadowable" deletion mechanism is not general enough: we need to be able to re-insert an entry when a prior one had been deleted before, but we can't rely on timestamps being strictly bigger on the re-insert. In that sense, this can be though as a similar problem than CASSANDRA-10965, though the solution there of a single flag is not enough since we can have to replace more than once.
> I think the proper solution would be to ship enough information to always be able to decide when a view deletion is shadowed. Which means that both liveness info (for updates) and shadowable deletion would need to ship the timestamp of any base table column that is part the view PK (so {{a}} in the example below).  It's doable (and not that hard really), but it does require a change to the sstable and intra-node protocol, which makes this a bit painful right now.
> But I'll also note that as CASSANDRA-1096 shows, the timestamp is not even enough since on equal timestamp the value can be the deciding factor. So in theory we'd have to ship the value of those columns (in the case of a deletion at least since we have it in the view PK for updates). That said, on that last problem, my preference would be that we start prioritizing CASSANDRA-6123 seriously so we don't have to care about conflicting timestamp anymore, which would make this problem go away.



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