Hi,

doing a quick scan over the thread two things that came into my mind:

Frist, did the restore copy the sstables to the right machines back? Node As 
data to node A and so on? 

Second, did you run full repairs on every node? Not just incremental ones which 
now is the default?

Also a look into debug.log is an option.

If all done already, nevermind.

Gesendet von meinem Windows 10 Phone

Von: Ryan Svihla
Gesendet: Donnerstag, 16. März 2017 18:57
An: user
Betreff: Re: Issue with Cassandra consistency in results

Depends actually, restore just restores what's there, so if only one node had a 
copy of the data then only one node had a copy of the data meaning quorum will 
still be wrong sometimes.

On Thu, Mar 16, 2017 at 1:53 PM, Arvydas Jonusonis 
<arvydas.jonuso...@gmail.com> wrote:
If the data was written at ONE, consistency is not guaranteed. ..but 
considering you just restored the cluster, there's a good chance something else 
is off.

On Thu, Mar 16, 2017 at 18:19 srinivasarao daruna <sree.srin...@gmail.com> 
wrote:
Want to make read and write QUORUM as well. 


On Mar 16, 2017 1:09 PM, "Ryan Svihla" <r...@foundev.pro> wrote:
        Replication factor is 3, and write consistency is ONE and read 
consistency is QUORUM.

That combination is not gonna work well:

Write succeeds to NODE A but fails on node B,C

Read goes to NODE B, C

If you can tolerate some temporary inaccuracy you can use QUORUM but may still 
have the situation where

Write succeeds on node A a timestamp 1, B succeeds at timestamp 2
Read succeeds on node B and C at timestamp 1 

If you need fully race condition free counts I'm afraid you need to use SERIAL 
or LOCAL_SERIAL (for in DC only accuracy)

On Thu, Mar 16, 2017 at 1:04 PM, srinivasarao daruna <sree.srin...@gmail.com> 
wrote:
Replication strategy is SimpleReplicationStrategy.

Smith is : EC2 snitch. As we deployed cluster on EC2 instances.

I was worried that CL=ALL have more read latency and read failures. But won't 
rule out trying it.

Should I switch select count (*) to select partition_key column? Would that be 
of any help.?


Thank you 
Regards
Srini

On Mar 16, 2017 12:46 PM, "Arvydas Jonusonis" <arvydas.jonuso...@gmail.com> 
wrote:
What are your replication strategy and snitch settings?

Have you tried doing a read at CL=ALL? If it's an actual inconsistency issue 
(missing data), this should cause the correct results to be returned. You'll 
need to run a repair to fix the inconsistencies.

If all the data is actually there, you might have one or several nodes that 
aren't identifying the correct replicas.

Arvydas



On Thu, Mar 16, 2017 at 5:31 PM, srinivasarao daruna <sree.srin...@gmail.com> 
wrote:
Hi Team, 

We are struggling with a problem related to cassandra counts, after backup and 
restore of the cluster. Aaron Morton has suggested to send this to user list, 
so some one of the list will be able to help me. 

We are have a rest api to talk to cassandra and one of our query which fetches 
count is creating problems for us.

We have done backup and restore and copied all the data to new cluster. We have 
done nodetool refresh on the tables, and did the nodetool repair as well.

However, one of our key API call is returning inconsistent results. The result 
count is 0 in the first call and giving the actual values for later calls. The 
query frequency is bit high and failure rate has also raised considerably.

1) The count query has partition keys in it. Didnt see any read timeout or any 
errors from api logs.

2) This is how our code of creating session looks.

val poolingOptions = new PoolingOptions
    poolingOptions
      .setCoreConnectionsPerHost(HostDistance.LOCAL, 4)
      .setMaxConnectionsPerHost(HostDistance.LOCAL, 10)
      .setCoreConnectionsPerHost(HostDistance.REMOTE, 4)
      .setMaxConnectionsPerHost( HostDistance.REMOTE, 10)

val builtCluster = clusterBuilder.withCredentials(username, password)
      .withPoolingOptions(poolingOptions)
      .build()
val cassandraSession = builtCluster.get.connect()

val preparedStatement = 
cassandraSession.prepare(statement).setConsistencyLevel(ConsistencyLevel.QUORUM)
cassandraSession.execute(preparedStatement.bind(args :_*))

Query: SELECT count(*) FROM table_name WHERE parition_column=? AND 
text_column_of_clustering_key=? AND date_column_of_clustering_key<=? AND 
date_column_of_clustering_key>=?

3) Cluster configuration:

6 Machines: 3 seeds, we are using apache cassandra 3.9 version. Each machine is 
equipped with 16 Cores and 64 GB Ram.

        Replication factor is 3, and write consistency is ONE and read 
consistency is QUORUM.

4) cassandra is never down on any machine

5) Using cassandra-driver-core artifact with 3.1.1 version in the api.

6) nodetool tpstats shows no read failures, and no other failures.

7) Do not see any other issues from system.log of cassandra. We just see few 
warnings as below.

Maximum memory usage reached (512.000MiB), cannot allocate chunk of 1.000MiB
WARN  [ScheduledTasks:1] 2017-03-14 14:58:37,141 QueryProcessor.java:103 - 88 
prepared statements discarded in the last minute because cache limit reached 
(32 MB)
The first api call returns 0 and the api calls later gives right values.

Please let me know, if any other details needed.
Could you please have a look at this issue once and kindly give me your inputs? 
This issue literally broke the confidence on Cassandra from our business team.

Your inputs will be really helpful.

Thank You,
Regards, 
Srini






-- 
Thanks,
Ryan Svihla




-- 
Thanks,
Ryan Svihla

Reply via email to