Yes, the application in quite read heavy and the request pattern is bursty
too. Hence that big a request failure in such less time.
Also, nothing out of the ordinary in cfstats and proxyhistograms.

But there are Native-Transport-Requests dropped messages (Almost similar
stats on all the nodes) :

 /opt/evive/pkgs/apache-cassandra-3.0.13/bin/nodetool tpstats
>
> Pool Name                              Active   Pending      Completed
> Blocked    All time blocked
> ...
> Native-Transport-Requests         6         0                27777476
>     0              8262
>
> Message type           Dropped
> READ                     24260
>


*We have the below values for the following params:*
concurrent_reads: 96
native_transport_max_threads: 320
JVM_OPTS="$JVM_OPTS -Dcassandra.max_queued_native_transport_requests=3072"

Do you think it would be advisable to tune the number in the above params
to have lesser load on the node?


On Tue, 27 Jul 2021 at 20:13, Bowen Song <bo...@bso.ng> wrote:

> Wow, 15 seconds timeout? That's pretty long... You may want to check the 
> nodetool
> tpstats and make sure the NTP thread pool isn't blocking things.
>
> 16k read requests dropped in 5 seconds, or over 3k requests per second on
> a single node, is a bit suspicious. Does your read requests tend to be
> bursty?
> On 27/07/2021 15:32, Chahat Bhatia wrote:
>
> Yes, RF=6 for system auth. Sorry my bad.
>
>
> No, we are not using cassandra user for the application. We have a custom
> super user for our operational and administrative tasks and a separate role
> with needed perms for the application.
>
>>  role                      | super | login | options
>> ------------------------+-------+-------+---------
>>  EviveCassandra |  True |  True |        {}
>
>                   ccusr | False |  True |        {}
>
>
>
>
>    -  gc_warn_threshold_in_ms: 1000
>    - read_request_timeout_in_ms: 15000  // been the same since the start
>    - cross_node_timeout: false           // but we have ntp on each
>    server and the clocks are synced.
>
>    - On checking the logs a little more I found out these:
>
> INFO  [ScheduledTasks:1] 2021-07-26 07:21:33,279
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 289 for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-26 07:22:33,894
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 273 for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-27 06:12:03,988
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 160 for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-27 06:13:09,125
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 223 for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-27 06:13:44,212
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms: 72
>> for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-27 06:14:14,496
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 16036 for internal timeout and 0 for cross node timeout
>
> INFO  [ScheduledTasks:1] 2021-07-27 06:15:59,744
>> MessagingService.java:1013 - READ messages were dropped in last 5000 ms:
>> 287 for internal timeout and 0 for cross node timeout
>
>
>
>    - Also, I am checking cfstats and proxyhistorgrams is in progress,
>    will update incase anythings suspicious.
>
>
> On Tue, 27 Jul 2021 at 18:09, Bowen Song <bo...@bso.ng> <bo...@bso.ng>
> wrote:
>
>> Hello Chahat,
>>
>>
>> You haven't replied to the first point, are you using the "cassandra"
>> user?
>>
>>
>> The schema and your description don't quite match. When you said:
>>
>> *the system_auth  for 2 DCs :   **us-east*
>> * with 6 nodes (and RF=3) and ... *
>>
>> I assume you meant to say 6 nodes and RF=6? That sounds totally fine.
>>
>>
>> As of the GC logs, do you see any WARN level logs? You may also want to
>> double check the choice of the gc_warn_threshold_in_ms value is
>> reasonable. I'm a bit surprised to see the GC logs full of ~1.6ms GC
>> pauses. AFAIK, by default Cassandra only logs GC pauses longer than 200ms.
>> I have to assume that's the result of your custom script. You may want to
>> double check the script actually does what it says on the tin, just in case
>> it misses something.
>>
>>
>> Then, the other relevant questions.
>>
>> Do you use cross node timeout? If so, are the clocks in sync between all
>> servers?
>>
>> What is the timeout values in the cassandra.yaml file? (
>> read_request_timeout_in_ms, etc.) Are they reasonably long enough for
>> the corresponding request type to complete?
>>
>> Since you've only got 7 nodes, I'd also recommend you to check the nodetool
>> cfstats & nodetool cfhistograms output for the tables in the system_auth
>> keyspace on each node, and see if you can spot anything out of ordinary.
>>
>>
>> Regards,
>>
>> Bowen
>> On 27/07/2021 09:23, Chahat Bhatia wrote:
>>
>> Thanks for the prompt response.
>>
>> * Here is the system_schema.keyspaces entry:*
>>
>>                 system_auth |           True | {'class':
>>> 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'us-east': '6',
>>> 'us-east-backup': '1'}
>>>                      census |           True | {'class':
>>> 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'us-east': '3',
>>> 'us-east-backup': '1'}
>>
>>
>> So, the system_auth  for 2 DCs :   *us-east with 6 nodes (and RF=3) and
>> us-east-backup with a single node and 100% of the data.*
>> *Keyspace census* showing RF=3 for the main DC and RF=1 for the backup
>> site.
>>
>>
>> And for running the application, we have a user specifically created for
>>> that application itself and that user also has its appropriate permissions
>>> in cassandra to select, modify and delete from concerned tables. And it
>>> uses LOCAL_QUORUM for querying the data and the local-dc is set to
>>> 'us-east'.
>>
>>
>>
>> Also, there is no excessive GC for any of the nodes, we run a custom
>> script to trackthe GC stats (from the cassandra log itself) and output it.
>> Below is the output from the current running script and its similar for all
>> the servers:
>>
>> 2021-07-27 03:04:45,072 INFO gcstats:58 - Application Thread stop time
>>> 0.001656 seconds.
>>> 2021-07-27 03:04:45,080 INFO gcstats:58 - Application Thread stop time
>>> 0.001669 seconds.
>>> 2021-07-27 03:04:45,087 INFO gcstats:58 - Application Thread stop time
>>> 0.001601 seconds.
>>> 2021-07-27 03:04:45,095 INFO gcstats:58 - Application Thread stop time
>>> 0.001713 seconds.
>>> 2021-07-27 03:04:45,103 INFO gcstats:58 - Application Thread stop time
>>> 0.001586 seconds.
>>> 2021-07-27 03:04:45,110 INFO gcstats:58 - Application Thread stop time
>>> 0.001671 seconds.
>>> 2021-07-27 03:04:45,118 INFO gcstats:58 - Application Thread stop time
>>> 0.001691 seconds.
>>> 2021-07-27 03:04:45,127 INFO gcstats:58 - Application Thread stop time
>>> 0.001860 seconds.
>>> 2021-07-27 03:04:45,134 INFO gcstats:58 - Application Thread stop time
>>> 0.001630 seconds.
>>> 2021-07-27 03:04:45,141 INFO gcstats:58 - Application Thread stop time
>>> 0.001515 seconds.
>>> 2021-07-27 03:04:45,148 INFO gcstats:58 - Application Thread stop time
>>> 0.001533 seconds.
>>> 2021-07-27 03:04:45,156 INFO gcstats:58 - Application Thread stop time
>>> 0.001630 seconds.
>>> 2021-07-27 03:04:45,163 INFO gcstats:58 - Application Thread stop time
>>> 0.001577 seconds.
>>> 2021-07-27 03:04:45,170 INFO gcstats:58 - Application Thread stop time
>>> 0.001538 seconds.
>>> 2021-07-27 03:04:45,177 INFO gcstats:58 - Application Thread stop time
>>> 0.001615 seconds.
>>> 2021-07-27 03:04:45,186 INFO gcstats:58 - Application Thread stop time
>>> 0.001584 seconds.
>>> 2021-07-27 03:04:45,193 INFO gcstats:58 - Application Thread stop time
>>> 0.001558 seconds.
>>> 2021-07-27 03:04:45,200 INFO gcstats:58 - Application Thread stop time
>>> 0.001696 seconds.
>>> 2021-07-27 03:04:45,208 INFO gcstats:58 - Application Thread stop time
>>> 0.001658 seconds.
>>> 2021-07-27 03:04:45,215 INFO gcstats:58 - Application Thread stop time
>>> 0.001592 seconds.
>>> 2021-07-27 03:04:45,222 INFO gcstats:58 - Application Thread stop time
>>> 0.001618 seconds.
>>> 2021-07-27 03:05:08,907 INFO gcstats:58 - Application Thread stop time
>>> 0.001624 seconds.
>>> 2021-07-27 03:06:34,436 INFO gcstats:58 - Application Thread stop time
>>> 0.297773 seconds.
>>>
>>
>>
>>
>>
>>
>> On Tue, 27 Jul 2021 at 13:23, 'Bowen Song' via Infra Updates <
>> infra-upda...@goevive.com> wrote:
>>
>>> Hello Chahat,
>>>
>>>
>>> First, can you please make sure the Cassandra user used by the
>>> application is not "cassandra"? Because the "cassandra" user uses
>>> QUORUM consistency level to read the auth tables.
>>>
>>> Then, can you please make sure the replication strategy is set correctly
>>> for the system_auth namespace? I.e.: ensure the old DC is not present, and
>>> the new DC has sufficient number of replicas for fault tolerance.
>>>
>>> Finally, can you please check the GC logs, and make sure there isn't JVM
>>> GC issues, espicially long STW pauses?
>>>
>>>
>>> Regards,
>>>
>>> Bowen
>>>
>>>
>>> On 27/07/2021 08:34, Chahat Bhatia wrote:
>>>
>>> Hi Community,
>>>
>>> Context: We are running a cluster of 6 nodes in production with a RF=3
>>> in AWS.
>>> We recently moved from physical servers to cloud by adding a new DC and
>>> then removing the old one. Everything is working fine in all the other
>>> applications except this one.
>>>
>>>
>>> *As we recently started experiencing read timeouts in one of our
>>> production applications where the client threw  *
>>>
>>> Error An unexpected error occurred server side on ip-IP.ec2.internal:
>>>> com.google.common.util.concurrent.*UncheckedExecutionException*:
>>>> *com.google.common.util.concurrent.UncheckedExecutionException:**
>>>> java.lang.RuntimeException:
>>>> org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed out -
>>>> received only 0 responses.*
>>>> com.datastax.driver.core.exceptions.ServerError: An unexpected error
>>>> occurred server side  :
>>>> com.google.common.util.concurrent.UncheckedExecutionException:
>>>> com.google.common.util.concurrent.UncheckedExecutionException:
>>>> java.lang.RuntimeException:
>>>> org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed
>>>> out - received only 0 responses.
>>>
>>> at com.datastax.driver.core.exceptions.ServerError.copy(
>>>> ServerError.java:63) ~[cassandra-driver-core-3.3.0-shaded.jar!/:?] at
>>>> com.datastax.driver.core.exceptions.ServerError.copy(ServerError.java:
>>>> 25) ~[cassandra-driver-core-3.3.0-shaded.jar!/:?] at
>>>> com.datastax.driver.core.DriverThrowables.propagateCause(
>>>> DriverThrowables.java:37) ~[cassandra-driver-core-3.3.0-shaded.jar!/:?]
>>>> at com.datastax.driver.core.DefaultResultSetFuture.getUninterruptibly(
>>>> DefaultResultSetFuture.java:245) ~[cassandra-driver-core-3
>>>> .3.0-shaded.jar!/:?] at
>>>> com.datastax.driver.core.AbstractSession.execute(AbstractSession.java:
>>>> 68) ~[cassandra-driver-core-3.3.0-shaded.jar!/:? ]
>>>
>>>  ............ cntd
>>>
>>>
>>>
>>> com.datastax.driver.core.exceptions.ReadTimeoutException: Cassandra
>>>> timeout during read query at consistency LOCAL_QUORUM (2 responses were
>>>> required but only 1 replica responded)
>>>
>>>
>>>
>>>
>>>
>>> *And around the same time these were the errors on the server side (from
>>> cassandra logs): *
>>>
>>>
>>>>
>>>> *ERROR [RolesCacheRefresh:1] 2021-07-26 06:32:43,094
>>>> CassandraDaemon.java:207 - Exception in thread
>>>> Thread[RolesCacheRefresh:1,5,main] java.lang.RuntimeException:
>>>> org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed out -
>>>> received only 0 responses. *        at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:512)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRoles(CassandraRoleManager.java:280)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.RolesCache$1$1.call(RolesCache.java:135)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.RolesCache$1$1.call(RolesCache.java:130)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>> ~[na:1.8.0_131]
>>>>         at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>>>> ~[na:1.8.0_131]
>>>>         at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>>>> [na:1.8.0_131]
>>>>         at
>>>> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>>>> [apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
>>>> Caused by: org.apache.cassandra.exceptions.ReadTimeoutException:
>>>> Operation timed out - received only 0 responses.
>>>>         at
>>>> org.apache.cassandra.service.ReadCallback.awaitResults(ReadCallback.java:132)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.ReadCallback.get(ReadCallback.java:137)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.AbstractReadExecutor.get(AbstractReadExecutor.java:145)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.awaitResultsAndRetryOnDigestMismatch(StorageProxy.java:1715)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1664)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1605)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1524)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.db.SinglePartitionReadCommand$Group.execute(SinglePartitionReadCommand.java:955)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:263)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:224)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRoleFromTable(CassandraRoleManager.java:520)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:502)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>
>>>>
>>>>
>>>> *ERROR [PermissionsCacheRefresh:1] 2021-07-26 07:11:25,804
>>>> CassandraDaemon.java:207 - Exception in thread
>>>> Thread[PermissionsCacheRefresh:1,5,main] java.lang.RuntimeException:
>>>> org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed out -
>>>> received only 0 responses. *        at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:512)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.isSuper(CassandraRoleManager.java:304)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.Roles.hasSuperuserStatus(Roles.java:52)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.AuthenticatedUser.isSuper(AuthenticatedUser.java:71)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraAuthorizer.authorize(CassandraAuthorizer.java:76)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.PermissionsCache$1$1.call(PermissionsCache.java:136)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.PermissionsCache$1$1.call(PermissionsCache.java:131)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>> ~[na:1.8.0_131]
>>>>         at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>>>> ~[na:1.8.0_131]
>>>>         at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>>>> [na:1.8.0_131]
>>>>         at
>>>> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>>>> [apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
>>>> Caused by: org.apache.cassandra.exceptions.ReadTimeoutException:
>>>> Operation timed out - received only 0 responses.
>>>>         at
>>>> org.apache.cassandra.service.ReadCallback.awaitResults(ReadCallback.java:132)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.ReadCallback.get(ReadCallback.java:137)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.AbstractReadExecutor.get(AbstractReadExecutor.java:145)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.awaitResultsAndRetryOnDigestMismatch(StorageProxy.java:1715)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1664)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1605)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1524)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.db.SinglePartitionReadCommand$Group.execute(SinglePartitionReadCommand.java:955)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:263)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:224)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRoleFromTable(CassandraRoleManager.java:520)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>         at
>>>> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:502)
>>>> ~[apache-cassandra-3.0.13.jar:3.0.13]
>>>>
>>>
>>>
>>>
>>> *These are the values of these params in my configuration file  *
>>>
>>> permissions_validity_in_ms: 300000
>>>> permissions_update_interval_in_ms: 20000
>>>> roles_validity_in_ms: 300000
>>>> roles_update_interval_in_ms: 20000
>>>>
>>>
>>> This was not the case earlier and since this comes from a single app
>>> alone we are not sure if this is actually the issue. Can anyone please
>>> point out if these values are misconfigured and hence causing the issue or
>>> is it somewhere else we should be looking at?
>>>
>>> Any help would be appreciated.
>>>
>>> Thanks & Regards,
>>> Chahat.
>>>
>>>
>>
>> --
>> Thanks & Regards,
>> Chahat Bhatia
>> Systems Engineer
>> *Evive*
>> +91 7087629779
>>
>>
>
> --
> Thanks & Regards,
> Chahat Bhatia
> Systems Engineer
> *Evive*
> +91 7087629779
>
>

-- 
Thanks & Regards,
Chahat Bhatia
Systems Engineer
*Evive*
+91 7087629779

Reply via email to