cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Craig Nicholson (JIRA)" <j...@apache.org>
Subject [jira] [Created] (CASSANDRA-13547) Filtered materialized views missing data
Date Tue, 23 May 2017 14:45:04 GMT
Craig Nicholson created CASSANDRA-13547:
-------------------------------------------

             Summary: Filtered materialized views missing data
                 Key: CASSANDRA-13547
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13547
             Project: Cassandra
          Issue Type: Bug
          Components: Materialized Views
         Environment: Official Cassandra 3.10 Docker image (ID 154b919bf8ce).
            Reporter: Craig Nicholson
            Priority: Blocker


When creating a materialized view against a base table the materialized view does not always
reflect the correct data.

Using the following test schema:

{code:title=Schema|language=sql}
DROP KEYSPACE IF EXISTS test;
CREATE KEYSPACE test
  WITH REPLICATION = { 
   'class' : 'SimpleStrategy', 
   'replication_factor' : 1 
  };
CREATE TABLE test.table1 (
                id int,
                name text,
                enabled boolean,
                foo text,
                PRIMARY KEY (id, name));
CREATE MATERIALIZED VIEW test.table1_mv1 AS SELECT id, name, foo
                FROM test.table1
                WHERE id IS NOT NULL 
                AND name IS NOT NULL 
                AND enabled = TRUE
                PRIMARY KEY ((name), id);
CREATE MATERIALIZED VIEW test.table1_mv2 AS SELECT id, name, foo, enabled
                FROM test.table1
                WHERE id IS NOT NULL 
                AND name IS NOT NULL 
                AND enabled = TRUE
                PRIMARY KEY ((name), id);
{code}

When I insert a row into the base table the materialized views are updated appropriately.
(+)
{code:title=Insert row|language=sql}
cqlsh> INSERT INTO test.table1 (id, name, enabled, foo) VALUES (1, 'One', TRUE, 'Bar');
cqlsh> SELECT * FROM test.table1;

 id | name | enabled | foo
----+------+---------+-----
  1 |  One |    True | Bar

(1 rows)
cqlsh> SELECT * FROM test.table1_mv1;

 name | id | foo
------+----+-----
  One |  1 | Bar

(1 rows)
cqlsh> SELECT * FROM test.table1_mv2;

 name | id | enabled | foo
------+----+---------+-----
  One |  1 |    True | Bar

(1 rows)
{code}


Updating the record in the base table and setting enabled to FALSE will filter the record
from both materialized views. (+)
{code:title=Disable the row|language=sql}
cqlsh> UPDATE test.table1 SET enabled = FALSE WHERE id = 1 AND name = 'One';
cqlsh> SELECT * FROM test.table1;

 id | name | enabled | foo
----+------+---------+-----
  1 |  One |   False | Bar

(1 rows)
cqlsh> SELECT * FROM test.table1_mv1;

 name | id | foo
------+----+-----

(0 rows)
cqlsh> SELECT * FROM test.table1_mv2;

 name | id | enabled | foo
------+----+---------+-----

(0 rows)
{code}


However a further update to the base table setting enabled to TRUE should include the record
in both materialzed views, however only one view (table1_mv2) gets updated. (-)
It appears that only the view (table1_mv2) that returns the filtered column (enabled) is updated.
(-)
Additionally columns that are not part of the partiion or clustering key are not updated.
You can see that the foo column has a null value in table1_mv2. (-)
{code:title=Enable the row|language=sql}
cqlsh> UPDATE test.table1 SET enabled = TRUE WHERE id = 1 AND name = 'One';
cqlsh> SELECT * FROM test.table1;

 id | name | enabled | foo
----+------+---------+-----
  1 |  One |    True | Bar

(1 rows)
cqlsh> SELECT * FROM test.table1_mv1;

 name | id | foo
------+----+-----

(0 rows)
cqlsh> SELECT * FROM test.table1_mv2;

 name | id | enabled | foo
------+----+---------+------
  One |  1 |    True | null

(1 rows)
{code}




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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


Mime
View raw message