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

Andrés de la Peña commented on CASSANDRA-13412:
-----------------------------------------------

[~ebautistabar], I can reproduce the problem in 2.1 and 2.2 this way:
{code}
CREATE KEYSPACE k WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 3};

CREATE TABLE k.t (
  a text,
  b text,
  c text,
  d set<int>,
  e float,
  f text,
  g int,
  h double,
  j set<int>,
  k float,
  m set<text>,
  PRIMARY KEY (a, b, c)
);
CREATE INDEX ON k.t(c);

INSERT INTO k.t (a, b, c, h) VALUES ('test_a', 'test_b', 'test_c', 1);
UPDATE k.t USING TTL 10 SET h = 10 WHERE a = 'test_a' AND b = 'test_b' AND c = 
'test_c';
-- Wait 10 seconds

SELECT * FROM k.t WHERE a = 'test_a' AND b = 'test_b' AND c = 'test_c'; -- 1 row
SELECT * FROM k.t WHERE c = 'test_c'; -- 0 rows
{code}
Or, in a simpler way:
{code}
CREATE KEYSPACE k WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 1};

CREATE TABLE k.t (
    pk int,
    ck int,
    a int,
    b int,
    PRIMARY KEY (pk, ck)
);
CREATE INDEX ON k.t(ck);

INSERT INTO k.t (pk, ck, a, b) VALUES (1, 2, 3, 4);
UPDATE k.t USING TTL 10 SET b = 10 WHERE pk = 1 AND ck = 2;
-- Wait 10 seconds

SELECT * FROM k.t WHERE pk = 1 AND ck = 2; -- 1 row
SELECT * FROM k.t WHERE ck = 2; -- 0 rows
{code}
However, I can't reproduce the problem in 3.0.11. Could you please provide a 
sequence of insertions producing the failure in 3.0.11?

> Update of column with TTL results in secondary index not returning row
> ----------------------------------------------------------------------
>
>                 Key: CASSANDRA-13412
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13412
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Enrique Bautista Barahona
>            Assignee: Andrés de la Peña
>
> Cassandra versions: 2.2.3, 3.0.11
> 1 datacenter, keyspace has RF 3. Default consistency level.
> Steps:
> 1. I create these table and index.
> {code}
> CREATE TABLE my_table (
>     a text,
>     b text,
>     c text,
>     d set<int>,
>     e float,
>     f text,
>     g int,
>     h double,
>     j set<int>,
>     k float,
>     m set<text>,
>     PRIMARY KEY (a, b, c)
> ) WITH read_repair_chance = 0.0
>    AND dclocal_read_repair_chance = 0.1
>    AND gc_grace_seconds = 864000
>    AND bloom_filter_fp_chance = 0.01
>    AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }
>    AND comment = ''
>    AND compaction = { 'class' : 
> 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy' }
>    AND compression = { 'sstable_compression' : 
> 'org.apache.cassandra.io.compress.LZ4Compressor' }
>    AND default_time_to_live = 0
>    AND speculative_retry = '99.0PERCENTILE'
>    AND min_index_interval = 128
>    AND max_index_interval = 2048;
> CREATE INDEX my_index ON my_table (c);
> {code}
> 2. I have 9951 INSERT statements in a file and I run the following command to 
> execute them. The INSERT statements have no TTL and no consistency level is 
> specified.
> {code}
> cqlsh <ip> <port> -u <user> -f <file>
> {code}
> 3. I update a column filtering by the whole primary key, and setting a TTL. 
> For example:
> {code}
> UPDATE my_table USING TTL 30 SET h = 10 WHERE a = 'test_a' AND b = 'test_b' 
> AND c = 'test_c';
> {code}
> 4. After the time specified in the TTL I run the following queries:
> {code}
> SELECT * FROM my_table WHERE a = 'test_a' AND b = 'test_b' AND c = 'test_c';
> SELECT * FROM my_table WHERE c = 'test_c';
> {code}
> The first one returns the correct row with an empty h column (as it has 
> expired). However, the second query (which uses the secondary index on column 
> c) returns nothing.
> I've done the query through my app which uses the Java driver v3.0.4 and 
> reads with CL local_one, from the cql shell and from DBeaver 3.8.5. All 
> display the same behaviour. The queries are performed minutes after the 
> writes and the servers don't have a high load, so I think it's unlikely to be 
> a consistency issue.
> I've tried to reproduce the issue in ccm and cqlsh by creating a new keyspace 
> and table, and inserting just 1 row, and the bug doesn't manifest. This leads 
> me to think that it's an issue only present with not trivially small amounts 
> of data, or maybe present only after Cassandra compacts or performs whatever 
> maintenance it needs to do.



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

Reply via email to