cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From <Adeline....@thomsonreuters.com>
Subject RE: WriteTimeoutException with LOCAL_QUORUM
Date Wed, 07 Sep 2016 09:09:27 GMT
Hi  Romain,

Thank you very much for the detailed reply. I really appreciate it. Here comes more information
and questions☺

1.       It is not a typo, we are really upgrading from Cassandra 1.0.2 to Cassandra 3.4.
We rebuilt the data models using CQL3 and changed all the thrift based APIs to use CQL. You
said you prefer the last 3.0.x rather than 3.4, could you tell me the reason?

2.       We do see a lot of hinted handoffs in the Cassandra log, about every minute. But
we didn’t see any UnavailableException, does that  mean that our nodes are up all the time
but too busy to response the write in time? We see “DEBUG [GossipStage:1] 2016-09-01 08:38:48,595
FailureDetector.java:456 - Ignoring interval time of 2020536424 for /xxx.xxx.xxx.xxx” a
lot, does that mean anything?

3.       BTW, I think I made a mistake in the last mail, the max heap size of Cassandra is
4GB ,and we didn’t specific the memtable size in Cassandra.yaml, that means the memtable
size is 1GB(1/4 of the heap), right?

4.       We didn’t specific the “memtable_cleanup_threshold” and “memtable_flush_writers”
in Cassandra.yaml, so the “memtable_cleanup_threshold” should be 0.5

5.       The batch_size_warn_threshold_in_kb = 5, and we do use batches for a few tables,
but it is very small batch, only 2 or 3 statement, that should be fine, right?

6.       I run “iostat -xmt 1” on the Cassandra nodes , the iowait can raise high from
time to time , but it will get down really quick, is it normal? BTW, we are using virtual
machines for the testing now,

the performance of machines might not be very good.

[cid:image001.png@01D20925.9B4A6670]

7.
                * org.apache.cassandra.metrics:type=CommitLog,name=WaitingOnCommit
            - Count:2455
                * org.apache.cassandra.metrics:type=CommitLog,name=WaitingOnSegmentAllocation
            - Mean :184422.20800783538
            - 99thPercentile:4139110.981
            - Max: 1.4831182994E7
        8. about the tombstone warning. We found that in that table (the same table that having
writetimeout exception), there are many rows that have two columns with null value, which
will be treated as tombstone, that explains why the tombstone number is always the twice of
the live rows.  We decided to not to write these columns when there is no value for it. Thank
you for your reminding me that☺
[cid:image002.png@01D2092A.03D89E10]

Many thanks again.

--
Regards, Adeline


From: Romain Hardouin [mailto:romainh_ml@yahoo.fr]
Sent: Tuesday, September 06, 2016 6:30 PM
To: user@cassandra.apache.org
Cc: Yang, Ling (TR Technology & Ops)
Subject: Re: WriteTimeoutException with LOCAL_QUORUM

1) Is it a typo or did you really make a giant leap from C* 1.x to 3.4 with all the C*2.0
and C*2.1 upgrades? (btw if I were you, I would use the last 3.0.X)

2) Regarding NTR all time blocked (e.g. 26070160 from the logs), have a look to the patch
"max_queued_ntr_property.txt": https://issues.apache.org/jira/browse/CASSANDRA-11363)
   Then set -Dcassandra.max_queued_native_transport_requests=XXX to a value that works for
you.

3) Regarding write timeouts:
   - Are your writes idempotent? You can retry when a WriteTimeoutException is catched, see
IdempotenceAwareRetryPolicy.
   - We can see Hints in the logs => Do you monitor the frequency/number of hints? Do you
see some UnavailableException at the driver level?
        It means that some nodes are unreachable and even if it should trigger an UnavailableException,
it may also raise WriteTimeoutException if the coordinator of a request doesn't know yet that
the node is unreachable (see failure detector)
   - 4 GB of heap is very small and you have 19 tables. Add 40 system tables to this and you
have 59 tables that share 4 GB.
   - You are using batches for one/some table(s), right? Is it really required? Is is the
most used table?
   - What are the values of
        * memtable_cleanup_threshold
        * batch_size_warn_threshold_in_kb
   - What the IO wait status on the nodes? Try to correlate timeout exceptions with IO wait
load.
   - Commitlog and data are on separate devices?
   - What are the value of the following Mbean attributes on each nodes?
        * org.apache.cassandra.metrics:type=CommitLog,name=WaitingOnCommit
            - Count
        * org.apache.cassandra.metrics:type=CommitLog,name=WaitingOnSegmentAllocation
            - Mean
            - 99thPercentile
            - Max
   - Do you see MemtableFlushWriter blocked tasks on nodes? I see 0 on the logs but the node
may have been restarted (e.g. 18 hours of uptime on the nodetool info).

4) Did you notice that you have tombstones warning? e.g.:

    WARN  [SharedPool-Worker-48] 2016-09-01 06:53:19,453 ReadCommand.java:481 - Read 5000
live rows and 10000 tombstone cells for query SELECT * FROM pc_object_data_beta.vsc_data WHERE
rundate, vscid = 1472653906000, 111034565 LIMIT 5000 (see tombstone_warn_threshold)

The chances are high that your data model is not optimal. You should *really* fix this.

Best,

Romain

Le Mardi 6 septembre 2016 6h47, "Adeline.Pan@thomsonreuters.com<mailto:Adeline.Pan@thomsonreuters.com>"
<Adeline.Pan@thomsonreuters.com<mailto:Adeline.Pan@thomsonreuters.com>> a écrit
:



From: Pan, Adeline (TR Technology & Ops)
Sent: Tuesday, September 06, 2016 12:34 PM
To: 'user@cassandra.apache.org'
Cc: Yang, Ling (TR Technology & Ops)
Subject: FW: WriteTimeoutException with LOCAL_QUORUM

Hi All,
I hope you are doing well today, and I need your help.

We were using Cassandra 1 before, now we are upgrading  to Cassandra 3.4 . During the integration
test, we encountered “WriteTimeoutException”  very frequently (about every other minute),
the exception message is as below.  The exception trace is in the attach file.

Caused by: com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout during
write query at consistency LOCAL_QUORUM (2 replica were required but only 1 acknowledged the
write)



There is some information:
1.       It is a six nodes cluster, two data centers, and three nodes for each datacenter.
The consistency level we are using is LOCAL_QUORUM
2.       The node info
[BETA:xxxx@xxxx:/local/java/cassandra3/current]$ bin/nodetool -hlocalhost info
ID                     : ad077318-6531-498e-bf5a-14ac339d1a45
Gossip active          : true
Thrift active          : false
Native Transport active: true
Load                   : 23.47 GB
Generation No          : 1473065408
Uptime (seconds)       : 67180
Heap Memory (MB)       : 1679.57 / 4016.00
Off Heap Memory (MB)   : 10.34
Data Center            : dc1
Rack                   : rack1
Exceptions             : 0
Key Cache              : entries 32940, size 3.8 MB, capacity 100 MB, 2124114 hits, 2252348
requests, 0.943 recent hit rate, 14400 save period in seconds
Row Cache              : entries 0, size 0 bytes, capacity 0 bytes, 0 hits, 0 requests, NaN
recent hit rate, 0 save period in seconds
Counter Cache          : entries 0, size 0 bytes, capacity 50 MB, 0 hits, 0 requests, NaN
recent hit rate, 7200 save period in seconds
Token                  : (invoke with -T/--tokens to see all 256 tokens)

3.       We have increased the write_request_timeout_in_ms to 40000,  which didn’t work.
4.       The memtable size is 4GB.
5.       memtable_allocation_type: heap_buffers
6.       In the Cassandra server log, we found there are Native-Transport-Requests  pending
from time to time. (The server log piece is in attach file)
INFO  [ScheduledTasks:1] 2016-09-01 10:08:47,036 StatusLogger.java:52 - Pool Name        
                     Active   Pending      Completed   Blocked  All Time Blocked
INFO  [ScheduledTasks:1] 2016-09-01 10:08:47,043 StatusLogger.java:56 - Native-Transport-Requests
      128       134      300721823         6          28211424



What also need to mention is there is always 1 node will acknowledge the write .Could you
please help me with this situation? Any clue will be appreciated. Thank you very much!
-------------------------------------------------------------------------------------------
Adeline Pan
Senior Software Engineer

Thomson Reuters
Phone: 62674654

Adeline.pan@thomsonreuters.com<mailto:Adeline.pan@thomsonreuters.com>


Mime
View raw message