cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Oleg Anastasjev <olega...@gmail.com>
Subject Cassandra cluster does not tolerate single node failure good
Date Wed, 07 Apr 2010 15:35:08 GMT
Hello,

I am doing some tests of cassandra clsuter behavior on several failure
scenarios.
And i am stuck with the very 1st test - what happens, if 1 node of cluster
becomes unavailable.
I have 4 4gb nodes loaded with write mostly test. Replication Factor is 2.
Normally it works at the rate about 12000 ops/second.
After a while, I shutdown node 4.
And whole cluster's performance drops down to 60 (yes, two hundred times
slower!) ops per second.
I checked this on both 0.5.0 and 0.5.1 versions.

This is my ring after shutdown:
Address       Status     Load          Range            Ring
                                       1276058875953
62.85.54.46   Up         119.03 MB     0                |<--|
62.85.54.47   Up         118.76 MB     4253529586511    |   |
62.85.54.48   Up         103.95 MB     8507059173023    |   |
62.85.54.49   Down       0 bytes       1276058875953    |-->|

After doing a bit of investigation, i found, that 62.85.54.46 and 62.85.54.47
started to starve in row mutation stage:
46:
ROW-MUTATION-STAGE               32       313        1875089
47:
ROW-MUTATION-STAGE               32      3042        1872123
but 48 is not:
ROW-MUTATION-STAGE                0         0        1668532

All these mutations go to HintsColumnFamily -
cfstats shows actility in this CF only for 46 and 47 nodes:
Keyspace: system
        Read Count: 0
        Read Latency: NaN ms.
        Write Count: 4953
        Write Latency: 386.766 ms.
        Pending Tasks: 0
                Column Family: LocationInfo
                Memtable Columns Count: 0
                Memtable Data Size: 0
                Memtable Switch Count: 1
                Read Count: 0
                Read Latency: NaN ms.
                Write Count: 0
                Write Latency: NaN ms.
                Pending Tasks: 0

                Column Family: HintsColumnFamily
                Memtable Columns Count: 173506
                Memtable Data Size: 1648344
                Memtable Switch Count: 1
                Read Count: 0
                Read Latency: NaN ms.
                Write Count: 4954
                Write Latency: 387.473 ms.
                Pending Tasks: 0
please note enormously slow write latency.

Interesting, that issuing "nodeprobe flush system" command to 46 and 47 nodes
speedup processing for a short period of time, but then it quickly returns
back to 66 ops/second.


I suspect, that these nodes create very much subcolumns in supercolumn of
CF HintsColumnFamily in memory table.

Investigating further,I found that when number of subcolumns becomes large in
single supercolumn, memory additions to it become very slow, probably because
of ConcurrentSkipListMap.size() -inspecting jstack outpunt I found, that
ROW-MUTATION threads are all waiting for a single thread
with the following stack trace:
java.util.concurrent.ConcurrentSkipListMap.size(ConcurrentSkipListMap.java:1715)
at org.apache.cassandra.db.SuperColumn.getObjectCount(SuperColumn.java:230)
at org.apache.cassandra.db.ColumnFamily.getColumnCount(ColumnFamily.java:148)
at org.apache.cassandra.db.Memtable.resolve(Memtable.java:159)
- locked <0x00002aaabe52fc98> (a java.lang.Object)
at org.apache.cassandra.db.Memtable.put(Memtable.java:143)
at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:478)
at org.apache.cassandra.db.Table.apply(Table.java:438)
at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:203)
at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(
RowMutationVerbHandler.java:58)
at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:38)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(
ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(
ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:619)

Can I do something to have cassandra cluster to tolerate single node failure
better ?





Mime
View raw message