Hi Shalom,

Thanks very much for the response!

 We are only using batches on one Cassandra partition to improve
performance. Batches are NEVER used in this app across Cassandra partition.
And if you look at the trace
messages I showed, there is only one statement per batch anyway.

In fact, what I see in the trace is that the responses to the writes may be
being held up by the reads. Here is a more complete example which is
consistent
across nodes. We are using datastax client 3.1.2. Note that all the
requests appear to be processed on nio-worker-5 which is suggesting that
this may be all on the one connection
(even though I can see two connections to each C* server from each client):



*2018-07-20 05:32:43,185 [luster1-nio-worker-5] [          ] [
       ] [                    ] (         core.QueryLogger.SLOW) DEBUG   -
[cluster1] [/10.123.4.52:9042 <http://10.123.4.52:9042>] Query too slow,
took 9322 ms: [2 bound values] select a, b, c, d from <table name1> where
token(a)>? and token(a)<=?; << slow read2018-07-20 05:32:43,185
[luster1-nio-worker-5] [          ] [                    ] [
     ] (         core.QueryLogger.SLOW) DEBUG   - [cluster1]
[/10.123.4.52:9042 <http://10.123.4.52:9042>] Query too slow, took 5950 ms:
[1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table name2> (a, b,
c, d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; << write response
received immediately after the read2018-07-20 05:32:43,185
[luster1-nio-worker-5] [          ] [                    ] [
     ] (         core.QueryLogger.SLOW) DEBUG   - [cluster1]
[/10.123.4.52:9042 <http://10.123.4.52:9042>] Query too slow, took 511 ms:
[1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table name2> (a, b,
c, d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; << write response
received immediately after the read*
2018-07-20 05:32:43,607 [luster1-nio-worker-5] [          ] [
     ] [                    ] (       core.QueryLogger.NORMAL) DEBUG   -
[cluster1] [/10.123.4.52:9042] Query completed normally, took 33 ms: [2
bound values] select CustomerID, ds_, data_, AudienceList from
data.customer_b01be157931bcbfa32b7f240a638129d where token(CustomerID)>?
and token(CustomerID)<=?; << normal read
2018-07-20 05:32:45,938 [luster1-nio-worker-5] [          ] [
     ] [                    ] (         core.QueryLogger.SLOW) DEBUG   -
[cluster1] [/10.123.4.52:9042] Query too slow, took 1701 ms: [2 bound
values] select a, b, c, d from <table name1> where token(a)>? and
token(a)<=?; << slow read
2018-07-20 05:32:46,257 [luster1-nio-worker-5] [          ] [
     ] [                    ] (       core.QueryLogger.NORMAL) DEBUG   -
[cluster1] [/10.123.4.52:9042] Query completed normally, took 0 ms: [1
statements, 6 bound values] BEGIN BATCH INSERT INTO <table name2> (a, b, c,
d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; << normal write – no
overlap with the read
2018-07-20 05:32:46,336 [luster1-nio-worker-5] [          ] [
     ] [                    ] (       core.QueryLogger.NORMAL) DEBUG   -
[cluster1] [/10.123.4.52:9042] Query completed normally, took 30 ms: [2
bound values] select a, b, c, d from <table name1> where token(a)>? and
token(a)<=?; << normal read

*2018-07-20 05:32:48,622 [luster1-nio-worker-5] [          ] [
       ] [                    ] (         core.QueryLogger.SLOW) DEBUG   -
[cluster1] [/10.123.4.52:9042 <http://10.123.4.52:9042>] Query too slow,
took 1626 ms: [2 bound values] select select a, b, c, d from <table name1>
where token(a)>? and token(a)<=?; << slow read2018-07-20 05:32:48,622
[luster1-nio-worker-5] [          ] [                    ] [
     ] (         core.QueryLogger.SLOW) DEBUG   - [cluster1]
[/10.123.4.52:9042 <http://10.123.4.52:9042>] Query too slow, took 425 ms:
[1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table name2> (a, b,
c, d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; << write appears
immediately after the read*

I would be suggesting some sort of bug on the client holding up the
thread...but I don't know why I would only have a problem on one C* node at
any one time (the clients process reads and writes to other nodes at the
same time without delays).

thanks in advance,
Gareth


On Sun, Jul 22, 2018 at 4:12 AM, shalom sagges <shalomsag...@gmail.com>
wrote:

> Hi Gareth,
>
> If you're using batches for multiple partitions, this may be the root
> cause you've been looking for.
>
> https://inoio.de/blog/2016/01/13/cassandra-to-batch-or-not-to-batch/
>
> If batches are optimally used and only one node is misbehaving, check if
> NTP on the node is properly synced.
>
> Hope this helps!
>
>
> On Sat, Jul 21, 2018 at 9:31 PM, Gareth Collins <
> gareth.o.coll...@gmail.com> wrote:
>
>> Hello,
>>
>> We are running Cassandra 2.1.14 in AWS, with c5.4xlarge machines
>> (initially these were m4.xlarge) for our cassandra servers and
>> m4.xlarge for our application servers. On one of our clusters having
>> problems we have 6 C* nodes and 6 AS nodes (two nodes for C*/AS in
>> each availability zone).
>>
>> In the deployed application it seems to be a common use-case to one of
>> the following. These use cases are having periodic errors:
>> (1) Copy one Cassandra table to another table using the application
>> server.
>> (2) Export from a Cassandra table to file using the application server.
>>
>> The application server is reading from the table via token range, the
>> token range queries being calculated to ensure the whole token range
>> for a query falls on the same node. i.e. the query looks like this:
>>
>> select * from <table> where token(key) > ? and token(key) <= ?
>>
>> This was probably initially done on the assumption that the driver
>> would be able to figure out which nodes contained the data. As we
>> realized now the driver only supports routing to the right node if the
>> partition key is defined in the where clause.
>>
>> When we do the read we are doing a lot of queries in parallel to
>> maximize performance. I believe when the copy is being run there are
>> currently 5 threads per machine doing the copy for a max of 30
>> concurrent read requests across the cluster.
>>
>> Specifically these tasks been periodically having a few of these errors:
>>
>> INFO  [ScheduledTasks:1] 2018-07-13 20:03:20,124
>> MessagingService.java:929 - REQUEST_RESPONSE messages were dropped in
>> last 5000 ms: 1 for internal timeout and 0 for cross node timeout
>>
>> Which are causing errors in the read by token range queries.
>>
>> Running "nodetool settraceprobability 1" and running the test when
>> failing we could see that this timeout would occur when using a
>> coordinator on the read query (i.e. the co-ordinator sent the message
>> but didn't get a response to the query from the other node within the
>> time limit). We were seeing these timeouts periodically even if we set
>> the timeouts to 60 seconds.
>>
>> As I mentioned at the beginning we had initially been using m4.xlarge
>> for our Cassandra servers. After discussion with AWS it was suggested
>> that we could be hitting performance limits (i.e. either network or
>> disk - I believe more likely network as I didn't see the disk getting
>> hit very hard) so we upgraded the Cassandra servers and everything was
>> fine for a while.
>>
>> But then the problems started to re-occur recently...pretty
>> consistently failing on these copy or export jobs running overnight.
>> Having looked at resource usage statistics graphs it appeared that the
>> C* servers were not heavily loaded at all (the app servers were being
>> maxed out) and I did not see any significant garbage collections in
>> the logs that could explain the delays.
>>
>> As a last resort I decided to turn up the logging on the server and
>> client, datastax client set to debug and server set to the following
>> logs via nodetool...the goal being to maximize logging while cutting
>> out the very verbose stuff (e.g. Message.java appears to print out the
>> whole message in 2.1.14 when put into debug -> it looks like that was
>> moved to trace in a later 2.1.x release):
>>
>> bin/nodetool setlogginglevel org.apache.cassandra.tracing.Tracing INFO
>> bin/nodetool setlogginglevel org.apache.cassandra.transport.Message INFO
>> bin/nodetool setlogginglevel org.apache.cassandra.db.ColumnFamilyStore
>> DEBUG
>> bin/nodetool setlogginglevel org.apache.cassandra.gms.Gossiper DEBUG
>> bin/nodetool setlogginglevel
>> org.apache.cassandra.db.filter.SliceQueryFilter DEBUG
>> bin/nodetool setlogginglevel
>> org.apache.cassandra.service.pager.AbstractQueryPager INFO
>> bin/nodetool setlogginglevel org.apache.cassandra TRACE
>>
>> Of course when we did this (as part of turning on the logging the
>> application servers were restarted) the problematic export to file
>> jobs which had failed every time for the last week succeeded and ran
>> much faster than they had run usually (47 minutes vs 1 1/2 hours) so I
>> decided to look for the biggest delay (which turned out to be ~9
>> seconds and see what I could find in the log - outside of this time,
>> the response times were up to perhaps 20ms). Here is what I found:
>>
>> (1) Only one Cassandra node had delays at a time.
>>
>> (2) On the Cassandra node that did had delays there was no significant
>> information from the GCInspector (the system stopped processing client
>> requests between 05:32:33 - 05:32:43). If anything it confirmed my
>> belief that the system was lightly loaded
>>
>> DEBUG [Service Thread] 2018-07-20 05:32:25,559 GCInspector.java:260 -
>> ParNew GC in 8ms.  CMS Old Gen: 2879689192 -> 2879724544; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 8701480 -> 12462936
>> DEBUG [Service Thread] 2018-07-20 05:32:26,792 GCInspector.java:260 -
>> ParNew GC in 9ms.  CMS Old Gen: 2879724544 -> 2879739504; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 12462936 -> 9463872
>> DEBUG [Service Thread] 2018-07-20 05:32:29,227 GCInspector.java:260 -
>> ParNew GC in 9ms.  CMS Old Gen: 2879739504 -> 2879755872; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 9463872 -> 10860856
>> DEBUG [Service Thread] 2018-07-20 05:32:33,190 GCInspector.java:260 -
>> ParNew GC in 8ms.  CMS Old Gen: 2879755872 -> 2879780816; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 10860856 -> 13637832
>> DEBUG [Service Thread] 2018-07-20 05:32:46,128 GCInspector.java:260 -
>> ParNew GC in 8ms.  CMS Old Gen: 2879780816 -> 2879807064; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 13637832 -> 7992928
>> DEBUG [Service Thread] 2018-07-20 05:32:51,001 GCInspector.java:260 -
>> ParNew GC in 9ms.  CMS Old Gen: 2879807064 -> 2879854128; Par Eden
>> Space: 335544320 -> 0; Par Survivor Space: 7992928 -> 10300800
>>
>> (3) The Cassandra node in question was still responding to MUTATION
>> requests (there are a small number of inserts going on at the same
>> time as this flow - probably about 40 every 10 seconds) and poll
>> requests from Gossip requests from other servers. When the problematic
>> node polled other servers it still reported a response being received
>> within at most a couple of milliseconds:
>>
>> TRACE [GossipTasks:1] 2018-07-20 05:32:36,043
>> MessagingService.java:693 - /10.123.4.xx sending GOSSIP_DIGEST_SYN to
>> 796502@/10.123.4.yy
>> TRACE [GossipStage:1] 2018-07-20 05:32:36,043
>> GossipDigestAckVerbHandler.java:41 - Received a GossipDigestAckMessage
>> from /10.123.4.yy
>>
>> (4) The "bad" Cassandra node was even processing MUTATION requests -
>> it just wasn't sending any responses back until after 05:32:43. e.g.
>> this is one of many occurring during the stop period:
>>
>> TRACE [SharedPool-Worker-1] 2018-07-20 05:32:36,596
>> MessagingService.java:693 - /10.123.4.xx sending MUTATION to
>> 796507@/10.123.4.zz
>> <<< TRACE [SharedPool-Worker-1] 2018-07-20 05:32:36,595
>> MessagingService.java:693 - /10.123.4.zz sending REQUEST_RESPONSE to
>> 796507@/10.123.4.52
>>
>> From the AS nodes:
>>
>> 2018-07-20 05:32:43,084 [luster1-nio-worker-5] [          ] [
>>           ] [                    ] (         core.QueryLogger.SLOW)
>> DEBUG   - [cluster1] [/10.123.4.52:9042] Query too slow, took 1731 ms:
>> [1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table>
>> (a,b,c,d,e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH;  >>> Must
>> have been sent ~05:32:41
>> 2018-07-20 05:32:43,049 [luster1-nio-worker-5] [          ] [
>>           ] [                    ] (         core.QueryLogger.SLOW)
>> DEBUG   - [cluster1] [/10.123.4.52:9042] Query too slow, took 4411 ms:
>> [1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table> (a, b,
>> c, d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; >>> Must
>> have been sent ~05:32:39
>> 2018-07-20 05:32:43,185 [luster1-nio-worker-5] [          ] [
>>           ] [                    ] (         core.QueryLogger.SLOW)
>> DEBUG   - [cluster1] [/10.123.4.52:9042] Query too slow, took 5950 ms:
>> [1 statements, 6 bound values] BEGIN BATCH INSERT INTO <table> (a, b,
>> c, d, e) VALUES (?, ?, ?, ?, ?) using ttl ?; APPLY BATCH; >>> Must
>> have been sent ~05:32:37
>>
>>
>>
>> I guess the question I have here is - what could I be missing which
>> could be causing this issue?:
>>
>> (a) Could there still be network limits between our AS and C* servers?
>> Would it be reasonable to try to update our AS servers? I understand
>> that m4.xlarge have much lower network throughputs than the c5.4xlarge
>> servers.
>>
>> (b) Could a potential bug in the AS screw up Cassandra? At least when
>> I have been looking at the past at netstat (before these servers were
>> upgraded - I only have very limited access to the specific servers now
>> having the problem), I could see big send queues building up on
>> Cassandra. I am wondering if there is a bug in the AS causing send
>> queues to the AS to build up, could this potentially cause Cassandra
>> threads to block and eventually cause timeouts (e.g. AS not reading
>> fast enough causes Cassandra processing threads to block which
>> eventually stops Cassandra from having threads available to process
>> internode requests and thus causing timeouts)? Or is this just not
>> possible?
>>
>> (c) Could this be related to a bug in Cassandra that is already fixed
>> in a later version of 2.1? I know there were some issues with netty
>> like this:
>>
>> https://github.com/netty/netty/issues/5486
>> https://github.com/netty/netty/pull/5506
>>
>> which was covered when netty was upgraded to 4.0.44 for
>> CASSANDRA-13114. I can't believe we are getting close to the native
>> transport requests limit (CASSANDRA-11363), especially when we are
>> only exporting to file (as there should be a maximum of around 30
>> requests outstanding roughly per cluster, not counting the occasional
>> insert from other tasks).
>>
>> I am trying very hard to find a reason to justify an upgrade as I know
>> staying on an older point release is problematic.
>>
>> (d) Is what we are doing just not a very good idea...and we are just
>> asking for problems by doing it this way? i.e. should be exporting a
>> different way...or perhaps reduce the parallelizing of these requests
>> (I think the thread count was reduced in the past before the C* server
>> upgrade and we still had problems, but I could always try again).
>>
>> Any ideas/suggestions are greatly appreciated.
>>
>> thanks in advance,
>> Gareth Collins
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
>> For additional commands, e-mail: user-h...@cassandra.apache.org
>>
>>
>

Reply via email to