cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jeff Jirsa <jji...@gmail.com>
Subject Re: Cassandra: Inconsistent data on reads (LOCAL_QUORUM)
Date Sun, 14 Oct 2018 17:55:17 GMT
If this is 2.1 AND you do deletes AND you have a non-zero number of failed writes (timeouts),
it’s possibly short reads

3.0 fixes this ( https://issues.apache.org/jira/browse/CASSANDRA-12872 ), it won’t be backported
to 2.1 because it’s a significant change to how reads are executed


-- 
Jeff Jirsa


> On Oct 13, 2018, at 7:24 PM, Naik, Ninad <ninad.naik@epsilon.com> wrote:
> 
> Thanks Maitrayee. I should have mentioned this as one of the things we verified. The
clocks on cassandra nodes are in sync. 
> From: maitrayee shah <kooljava2@yahoo.com.INVALID>
> Sent: Friday, October 12, 2018 6:40:25 PM
> To: user@cassandra.apache.org
> Subject: Re: Cassandra: Inconsistent data on reads (LOCAL_QUORUM)
>  
> [ This email has been sent from a source external to Epsilon. Please use caution when
clicking links or opening attachments. ]
> 
> We have seen inconsistent read if the clock on the nodes are not in sync. 
> 
> 
> Thank you 
> 
> Sent from my iPhone
> 
> On Oct 12, 2018, at 1:50 PM, Naik, Ninad <ninad.naik@epsilon.com> wrote:
> 
>> Hello,
>> 
>> We're seeing inconsistent data while doing reads on cassandra. Here are the details:
>> 
>> It's is a wide column table. The columns can be added my multiple machines, and read
by multiple machines. The time between writes and reads are in minutes, but sometimes can
be in seconds. Writes happen every 2 minutes.
>> 
>> Now, while reading we're seeing the following cases of inconsistent reads:
>> 
>> One column was added. If a read was done after the column was added (20 secs to 2
minutes after the write), Cassandra returns no data. As if the key doesn't exist. If the application
retries, it gets the data.
>> A few columns exist for a row key. And a new column 'n' was added. Again, a read
happens a few minutes after the write. This time, only the latest column 'n' is returned.
In this case the app doesn't know that the data is incomplete so it doesn't retry. If we manually
retry, we see all the columns.
>> A few columns exist for a row key. And a new column 'n' is added. When a read happens
after the write, all columns but 'n' are returned.
>> Here's what we've verified:
>> 
>> Both writes and reads are using 'LOCAL_QUORUM' consistency level.
>> The replication is within local data center. No remote data center is involved in
the read or write.
>> During the inconsistent reads, none of the nodes are undergoing GC pauses
>> There are no errors in cassandra logs
>> Reads always happen after the writes.
>> A few other details: Cassandra version: 2.1.9 DataStax java driver version: 2.1.10.2
Replication Factor: 3
>> 
>> We don't see this problem in lower environments. We have seen this happen once or
twice last year, but since last few days it's happening quite frequently. On an average 2
inconsistent reads every minute.
>> 
>> Here's how the table definition looks like:
>> 
>> CREATE TABLE "MY_TABLE" (
>>   key text,
>>   sub_key text,
>>   value text,
>>   PRIMARY KEY ((key), sub_key)
>> ) WITH
>>   bloom_filter_fp_chance=0.010000 AND
>>   caching='{"keys":"ALL", "rows_per_partition":"NONE"}' AND
>>   comment='' AND
>>   dclocal_read_repair_chance=0.100000 AND
>>   gc_grace_seconds=864000 AND
>>   read_repair_chance=0.000000 AND
>>   default_time_to_live=0 AND
>>   speculative_retry='ALWAYS' AND
>>   memtable_flush_period_in_ms=0 AND
>>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>>   compression={'sstable_compression': 'LZ4Compressor'};
>> Please point us in the right direction. Thanks !
>> 
>>  
>> 
>> The information contained in this e-mail message and any attachments may be privileged
and confidential. If the reader of this message is not the intended recipient or an agent
responsible for delivering it to the intended recipient, you are hereby notified that any
review, dissemination, distribution or copying of this communication is strictly prohibited.
If you have received this communication in error, please notify the sender immediately by
replying to this e-mail and delete the message and any attachments from your computer.
>> 
> 
>  
> 
> The information contained in this e-mail message and any attachments may be privileged
and confidential. If the reader of this message is not the intended recipient or an agent
responsible for delivering it to the intended recipient, you are hereby notified that any
review, dissemination, distribution or copying of this communication is strictly prohibited.
If you have received this communication in error, please notify the sender immediately by
replying to this e-mail and delete the message and any attachments from your computer.

Mime
View raw message