You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Wayne <wa...@gmail.com> on 2010/08/19 20:48:06 UTC

Node OOM Problems

I am having some serious problems keeping a 6 node cluster up and running
and stable under load. Any help would be greatly appreciated.

Basically it always comes back to OOM errors that never seem to subside.
After 5 minutes or 3 hours of heavy load depending on settings one or more
nodes seem to go down with an OOM error. Upon restart the node tries to get
going but ends up with an OOM error again and again and again. This patterns
repeats and repeats with no way out. Once a node goes south on me deleting
everything and starting from scratch is my only option. I can never get it
to stay up again.

I will try to describe the latest in detail. I am running a migration from a
MySQL database with 24 processes against 6 nodes in python using the std
thrift client. Should be a piece of cake in cassandra but no matter what I
have tried it always seems to kill a node after not too long.

6 x Nodes with 8 cores, 8G heap, 24g physical
0.6.4
JVM Options for lower compaction priority
JVM 1.6.0_21-b06

After 3 hours one node hard crashed with an OOM error. See below for
messages.

I restart and then see nothing but MessageDeserializationTask & some
GCInspector messages.

 INFO [GC inspection] 2010-08-19 17:06:09,228 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             1         1
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,227
MessageDeserializationTask.java (line 47) dropping message (52,288ms past
timeout)
 INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,229
MessageDeserializationTask.java (line 47) dropping message (52,290ms past
timeout)
 INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 1         1


After the 2nd restart it dies/freezes completely without actually crashing
(kill -9 is required). I knew it was dead because nothing was logged for 30
minutes to the log and nodetool still registered the node as down.

After the 3rd reboot it comes back UP but then eventually crashes again. See
below for details.

I have tried using key or row cache but cache makes the problem happen in 5
minutes as there is less memory and the OOM happens almost immediately. I
have played with setting less concurrent readers and writers and now have it
set to 16/16 (read/write). I have tried to set the mmap_index_only with no
change as the issue is really a JVM issue. All other settings are
std/default.

Does anyone keep their nodes up under load over time? I have been working
with cassandra for a while now and still have yet to keep anything up under
load for very long. I know nothing about java, and frankly am starting to
wonder if I need to be a java programmer to use use cassandra/run in debug
mode. Any help would be greatly appreciated. These issues in the past made
me try to use hbase which was solid as a rock (comparatively) but SLOW and
too complicated...I came back again thinking .6.4 and soon .7 would be
better but it almost seems worse to me in terms of stability. Frankly next I
will have to look at other alternatives because cassandra is totally
unstable. I do not want to give up after all this time but I am out of
ideas.

HELP!

Messages prior to first crash:

 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:33:19,985
MessageDeserializationTask.java (line 47) dropping message (12,024ms past
timeout)
 INFO [GC inspection] 2010-08-19 16:33:19,985 GCInspector.java (line 143)
CONSISTENCY-MANAGER               0         0
 INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
LB-TARGET                         0         0
 INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
ROW-MUTATION-STAGE                1       167
 INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
MESSAGE-STREAMING-POOL            0         0
 INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
LOAD-BALANCER-STAGE               0         0
 INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
FLUSH-SORTER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             0         0
 INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
HINTED-HANDOFF-POOL               1         1
 INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 148)
CompactionManager               n/a         0
 INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578) InetAddress
/10.4.0.202 is now UP
 INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578) InetAddress
/10.4.0.203 is now UP
 INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
/10.4.0.201 is now UP
 INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
/10.4.0.204 is now UP
 INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
/10.4.0.205 is now UP
 INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254 ColumnFamilyStore.java
(line 357) Point.Time.Data.Value has reached its threshold; switching in a
fresh Memtable at
CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
position=15371920)
 INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254 ColumnFamilyStore.java
(line 609) Enqueuing flush of Memtable-Point.Time.Data.Value@951707146(3462840
bytes, 314574 operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:21,254 Memtable.java (line 148)
Writing Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:22,090 Memtable.java (line 162)
Completed flushing
/mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-362-Data.db
 INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289 ColumnFamilyStore.java
(line 357) Point.Data.Time.Value has reached its threshold; switching in a
fresh Memtable at
CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
position=25251401)
 INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289 ColumnFamilyStore.java
(line 609) Enqueuing flush of Memtable-Point.Data.Time.Value@142023252(6330574
bytes, 314746 operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,289 Memtable.java (line 148)
Writing Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,785 Memtable.java (line 162)
Completed flushing
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
 INFO [COMPACTION-POOL:1] 2010-08-19 16:33:23,787 CompactionManager.java
(line 246) Compacting
[org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db')]
 INFO [GC inspection] 2010-08-19 16:34:03,565 GCInspector.java (line 116) GC
for ConcurrentMarkSweep: 39651 ms, 414341376 reclaimed leaving 8139886200
used; max is 8700035072
 INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 137)
Pool Name                    Active   Pending
 INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 143)
STREAM-STAGE                      0         0
 INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
FILEUTILS-DELETE-POOL             0         0
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,567
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-358-Data.db
 INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
RESPONSE-STAGE                    0         0
 INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
ROW-READ-STAGE                    0         0
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,584
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/system/HintsColumnFamily-7-Data.db
 INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
LB-OPERATIONS                     0         0
 INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
GMFD                              0         1
 INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
MESSAGE-DESERIALIZER-POOL         0         0
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,585
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/system/HintsColumnFamily-8-Data.db
 INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
CONSISTENCY-MANAGER               0         0
 INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
LB-TARGET                         0         0
 INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
ROW-MUTATION-STAGE                8        49
 INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
MESSAGE-STREAMING-POOL            0         0
 INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
LOAD-BALANCER-STAGE               0         0
 INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
FLUSH-SORTER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             1         1
 INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
HINTED-HANDOFF-POOL               1         6
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,589
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-357-Data.db
 INFO [GC inspection] 2010-08-19 16:34:03,589 GCInspector.java (line 148)
CompactionManager               n/a         1
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,600
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-356-Data.db
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,603
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-359-Data.db
 INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
(line 320) Compacted to
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db.
41190813/36949030 bytes for 9048 keys.  Time: 1399ms.
 INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
(line 246) Compacting
[org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1009-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1017-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1026-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db')]
 INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line 116) GC
for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving 8326856720
used; max is 8700035072
 INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 137)
Pool Name                    Active   Pending
 INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
STREAM-STAGE                      0         0
 INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
FILEUTILS-DELETE-POOL             0         0
 INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
RESPONSE-STAGE                    0         0
 INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
ROW-READ-STAGE                    0         0
 INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
LB-OPERATIONS                     0         0
 INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
GMFD                              0         0
 INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
MESSAGE-DESERIALIZER-POOL         1         1
 INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
CONSISTENCY-MANAGER               0         0
 INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
LB-TARGET                         0         0
 INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
ROW-MUTATION-STAGE                4         1
 INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
MESSAGE-STREAMING-POOL            0         0
 INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
LOAD-BALANCER-STAGE               0         0
 INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
FLUSH-SORTER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             0         0
 INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 143)
HINTED-HANDOFF-POOL               1         6
 INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 148)
CompactionManager               n/a         1
 INFO [Timer-0] 2010-08-19 16:36:00,785 Gossiper.java (line 180) InetAddress
/10.4.0.205 is now dead.
 INFO [Timer-0] 2010-08-19 16:36:00,786 Gossiper.java (line 180) InetAddress
/10.4.0.204 is now dead.
 INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116) GC
for ConcurrentMarkSweep: 37122 ms, 157488 reclaimed leaving 8342836376 used;
max is 8700035072
 INFO [GC inspection] 2010-08-19 16:36:00,789 GCInspector.java (line 137)
Pool Name                    Active   Pending
 INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
STREAM-STAGE                      0         0
 INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
FILEUTILS-DELETE-POOL             0         0
 INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
RESPONSE-STAGE                    0         0
 INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
ROW-READ-STAGE                    0         0
 INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
LB-OPERATIONS                     0         0
 INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
GMFD                              0         0
 INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
MESSAGE-DESERIALIZER-POOL         0        82
 INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
CONSISTENCY-MANAGER               0         0
 INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
LB-TARGET                         0         0
 INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
ROW-MUTATION-STAGE                3       770
 INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
MESSAGE-STREAMING-POOL            0         0
 INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
LOAD-BALANCER-STAGE               0         0
 INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
FLUSH-SORTER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             0         0
 INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,796
MessageDeserializationTask.java (line 47) dropping message (44,076ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
MessageDeserializationTask.java (line 47) dropping message (44,077ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
MessageDeserializationTask.java (line 47) dropping message (44,077ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,798
MessageDeserializationTask.java (line 47) dropping message (44,078ms past
timeout)
 INFO [Timer-0] 2010-08-19 16:36:00,797 Gossiper.java (line 180) InetAddress
/10.4.0.203 is now dead.
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,797
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,805
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,808
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db
 INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,812
SSTableDeletingReference.java (line 104) Deleted
/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db
 INFO [GC inspection] 2010-08-19 16:36:00,833 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 0         0
 INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 143)
HINTED-HANDOFF-POOL               1         6
 INFO [GMFD:1] 2010-08-19 16:36:00,834 Gossiper.java (line 578) InetAddress
/10.4.0.204 is now UP
 INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 148)
CompactionManager               n/a         1
 INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578) InetAddress
/10.4.0.203 is now UP
 INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578) InetAddress
/10.4.0.205 is now UP
 INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898 ColumnFamilyStore.java
(line 357) Point.Time.Data.Value has reached its threshold; switching in a
fresh Memtable at
CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
position=41642302)
 INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898 ColumnFamilyStore.java
(line 609) Enqueuing flush of Memtable-Point.Time.Data.Value@543707511(4032380
bytes, 374579 operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:36:00,898 Memtable.java (line 148)
Writing Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
operations)
ERROR [Thread-121] 2010-08-19 16:36:00,911 CassandraDaemon.java (line 83)
Uncaught exception in thread Thread[Thread-121,5,main]
java.lang.OutOfMemoryError: Java heap space
    at
org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)



Messages prior to last crash.


 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:06:32,245
MessageDeserializationTask.java (line 47) dropping message (525,696ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,391
MessageDeserializationTask.java (line 47) dropping message (562,843ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:07:09,391 GCInspector.java (line 143)
LB-OPERATIONS                     0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
MessageDeserializationTask.java (line 47) dropping message (562,843ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:07:09,392 GCInspector.java (line 143)
GMFD                              0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
MessageDeserializationTask.java (line 47) dropping message (562,844ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,394
MessageDeserializationTask.java (line 47) dropping message (562,846ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,315
MessageDeserializationTask.java (line 47) dropping message (595,767ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
MessageDeserializationTask.java (line 47) dropping message (595,768ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
MessageDeserializationTask.java (line 47) dropping message (595,768ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:07:41,965 GCInspector.java (line 143)
MESSAGE-DESERIALIZER-POOL         1    567231
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,189
MessageDeserializationTask.java (line 47) dropping message (631,641ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:08:18,192 GCInspector.java (line 143)
CONSISTENCY-MANAGER               0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,193
MessageDeserializationTask.java (line 47) dropping message (631,645ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:18,355
MessageDeserializationTask.java (line 47) dropping message (691,807ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:09:18,356 GCInspector.java (line 143)
LB-TARGET                         0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:44,730
MessageDeserializationTask.java (line 47) dropping message (691,808ms past
timeout)
ERROR [COMPACTION-POOL:1] 2010-08-19 18:18:42,658 CassandraDaemon.java (line
83) Uncaught exception in thread Thread[COMPACTION-POOL:1,1,main]
java.util.concurrent.ExecutionException: java.lang.OutOfMemoryError: Java
heap space
    at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
    at java.util.concurrent.FutureTask.get(FutureTask.java:83)
    at
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:86)
    at
org.apache.cassandra.db.CompactionManager$CompactionExecutor.afterExecute(CompactionManager.java:577)
    at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888)
    at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
    at java.lang.Thread.run(Thread.java:619)
Caused by: java.lang.OutOfMemoryError: Java heap space
    at
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:84)
    at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:335)
    at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:284)
    at
org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:117)
    at
org.apache.cassandra.db.ColumnFamilySerializer.deserializeFromSSTable(ColumnFamilySerializer.java:160)
    at
org.apache.cassandra.io.IteratingRow.getColumnFamily(IteratingRow.java:79)
    at
org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:112)
    at
org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:41)
    at
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:73)
    at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
    at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
    at
org.apache.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183)
    at
org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94)
    at
org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:295)
    at
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:102)
    at
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:83)
    at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
    at java.util.concurrent.FutureTask.run(FutureTask.java:138)
    at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
    ... 2 more
ERROR [Thread-13] 2010-08-19 18:18:42,658 CassandraDaemon.java (line 83)
Uncaught exception in thread Thread[Thread-13,5,main]
java.lang.OutOfMemoryError: Java heap space
    at java.util.concurrent.FutureTask.<init>(FutureTask.java:64)
    at
java.util.concurrent.AbstractExecutorService.newTaskFor(AbstractExecutorService.java:58)
    at
java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:77)
    at
org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:73)
 INFO [GC inspection] 2010-08-19 18:18:42,658 GCInspector.java (line 143)
ROW-MUTATION-STAGE                3         3
 INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
MESSAGE-STREAMING-POOL            0         0
 INFO [COMPACTION-POOL:1] 2010-08-19 18:18:42,659 CompactionManager.java
(line 246) Compacting
[org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1071-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1072-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1073-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1074-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1075-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1076-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1077-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1078-Data.db')]
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,658
MessageDeserializationTask.java (line 47) dropping message (1,256,110ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
LOAD-BALANCER-STAGE               0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,667
MessageDeserializationTask.java (line 47) dropping message (1,256,119ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:18:42,960 GCInspector.java (line 143)
FLUSH-SORTER-POOL                 0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:18:42,962 GCInspector.java (line 143)
MEMTABLE-POST-FLUSHER             0         0
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,963
MessageDeserializationTask.java (line 47) dropping message (1,256,415ms past
timeout)
 INFO [GC inspection] 2010-08-19 18:18:42,965 GCInspector.java (line 143)
AE-SERVICE-STAGE                  0         0
 INFO [GC inspection] 2010-08-19 18:18:42,968 GCInspector.java (line 143)
FLUSH-WRITER-POOL                 0         0

Re: Node OOM Problems

Posted by Edward Capriolo <ed...@gmail.com>.
On Thu, Aug 19, 2010 at 4:13 PM, Wayne <wa...@gmail.com> wrote:
> We are using the random partitioner. The tokens we defined manually and data
> is almost totally equal among nodes, 15GB per node when the trouble started.
> System vitals look fine. CPU load is ~500% for java, iostats are low,
> everything for all practical purposes looks "normal". Cfstats reports ~1-2
> ms for a read and ~.015-.02 for a write. All cache is now turned off as that
> seems to cause issues a lot faster. In terms of the current load, it is
> single insert statements. 6 nodes and 4 processes pointed to each node. That
> is not blasting away in my book.
>
> We can go a lot higher with memory, but with an 8g heap an NO cache thought
> this was a good number. With cache we would go higher on the memory but we
> are trying to get past this issue before even using cache. The possibility
> of a rogue large row is one I never considered. The largest row on the other
> nodes is as much as 800megs. I can not get a cfstats reading on the bad node
> so do not know how big its largest row is. I will raise memory to 16gb and
> see if that makes a difference. I had though that the java heap sizes that
> high had issues on their own in term of GC.
>
> Thanks.
>
>
> On Thu, Aug 19, 2010 at 9:44 PM, Edward Capriolo <ed...@gmail.com>
> wrote:
>>
>> On Thu, Aug 19, 2010 at 2:48 PM, Wayne <wa...@gmail.com> wrote:
>> > I am having some serious problems keeping a 6 node cluster up and
>> > running
>> > and stable under load. Any help would be greatly appreciated.
>> >
>> > Basically it always comes back to OOM errors that never seem to subside.
>> > After 5 minutes or 3 hours of heavy load depending on settings one or
>> > more
>> > nodes seem to go down with an OOM error. Upon restart the node tries to
>> > get
>> > going but ends up with an OOM error again and again and again. This
>> > patterns
>> > repeats and repeats with no way out. Once a node goes south on me
>> > deleting
>> > everything and starting from scratch is my only option. I can never get
>> > it
>> > to stay up again.
>> >
>> > I will try to describe the latest in detail. I am running a migration
>> > from a
>> > MySQL database with 24 processes against 6 nodes in python using the std
>> > thrift client. Should be a piece of cake in cassandra but no matter what
>> > I
>> > have tried it always seems to kill a node after not too long.
>> >
>> > 6 x Nodes with 8 cores, 8G heap, 24g physical
>> > 0.6.4
>> > JVM Options for lower compaction priority
>> > JVM 1.6.0_21-b06
>> >
>> > After 3 hours one node hard crashed with an OOM error. See below for
>> > messages.
>> >
>> > I restart and then see nothing but MessageDeserializationTask & some
>> > GCInspector messages.
>> >
>> >  INFO [GC inspection] 2010-08-19 17:06:09,228 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             1         1
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,227
>> > MessageDeserializationTask.java (line 47) dropping message (52,288ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,229
>> > MessageDeserializationTask.java (line 47) dropping message (52,290ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 1         1
>> >
>> >
>> > After the 2nd restart it dies/freezes completely without actually
>> > crashing
>> > (kill -9 is required). I knew it was dead because nothing was logged for
>> > 30
>> > minutes to the log and nodetool still registered the node as down.
>> >
>> > After the 3rd reboot it comes back UP but then eventually crashes again.
>> > See
>> > below for details.
>> >
>> > I have tried using key or row cache but cache makes the problem happen
>> > in 5
>> > minutes as there is less memory and the OOM happens almost immediately.
>> > I
>> > have played with setting less concurrent readers and writers and now
>> > have it
>> > set to 16/16 (read/write). I have tried to set the mmap_index_only with
>> > no
>> > change as the issue is really a JVM issue. All other settings are
>> > std/default.
>> >
>> > Does anyone keep their nodes up under load over time? I have been
>> > working
>> > with cassandra for a while now and still have yet to keep anything up
>> > under
>> > load for very long. I know nothing about java, and frankly am starting
>> > to
>> > wonder if I need to be a java programmer to use use cassandra/run in
>> > debug
>> > mode. Any help would be greatly appreciated. These issues in the past
>> > made
>> > me try to use hbase which was solid as a rock (comparatively) but SLOW
>> > and
>> > too complicated...I came back again thinking .6.4 and soon .7 would be
>> > better but it almost seems worse to me in terms of stability. Frankly
>> > next I
>> > will have to look at other alternatives because cassandra is totally
>> > unstable. I do not want to give up after all this time but I am out of
>> > ideas.
>> >
>> > HELP!
>> >
>> > Messages prior to first crash:
>> >
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:33:19,985
>> > MessageDeserializationTask.java (line 47) dropping message (12,024ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 16:33:19,985 GCInspector.java (line
>> > 143)
>> > CONSISTENCY-MANAGER               0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line
>> > 143)
>> > LB-TARGET                         0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line
>> > 143)
>> > ROW-MUTATION-STAGE                1       167
>> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line
>> > 143)
>> > MESSAGE-STREAMING-POOL            0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line
>> > 143)
>> > LOAD-BALANCER-STAGE               0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line
>> > 143)
>> > FLUSH-SORTER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line
>> > 143)
>> > HINTED-HANDOFF-POOL               1         1
>> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line
>> > 148)
>> > CompactionManager               n/a         0
>> >  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.202 is now UP
>> >  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.203 is now UP
>> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.201 is now UP
>> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.204 is now UP
>> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.205 is now UP
>> >  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254
>> > ColumnFamilyStore.java
>> > (line 357) Point.Time.Data.Value has reached its threshold; switching in
>> > a
>> > fresh Memtable at
>> >
>> > CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
>> > position=15371920)
>> >  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254
>> > ColumnFamilyStore.java
>> > (line 609) Enqueuing flush of
>> > Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
>> > operations)
>> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:21,254 Memtable.java (line
>> > 148)
>> > Writing Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
>> > operations)
>> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:22,090 Memtable.java (line
>> > 162)
>> > Completed flushing
>> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-362-Data.db
>> >  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289
>> > ColumnFamilyStore.java
>> > (line 357) Point.Data.Time.Value has reached its threshold; switching in
>> > a
>> > fresh Memtable at
>> >
>> > CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
>> > position=25251401)
>> >  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289
>> > ColumnFamilyStore.java
>> > (line 609) Enqueuing flush of
>> > Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
>> > operations)
>> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,289 Memtable.java (line
>> > 148)
>> > Writing Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
>> > operations)
>> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,785 Memtable.java (line
>> > 162)
>> > Completed flushing
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
>> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:33:23,787 CompactionManager.java
>> > (line 246) Compacting
>> >
>> > [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db')]
>> >  INFO [GC inspection] 2010-08-19 16:34:03,565 GCInspector.java (line
>> > 116) GC
>> > for ConcurrentMarkSweep: 39651 ms, 414341376 reclaimed leaving
>> > 8139886200
>> > used; max is 8700035072
>> >  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line
>> > 137)
>> > Pool Name                    Active   Pending
>> >  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line
>> > 143)
>> > STREAM-STAGE                      0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line
>> > 143)
>> > FILEUTILS-DELETE-POOL             0         0
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,567
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-358-Data.db
>> >  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line
>> > 143)
>> > RESPONSE-STAGE                    0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line
>> > 143)
>> > ROW-READ-STAGE                    0         0
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,584
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/system/HintsColumnFamily-7-Data.db
>> >  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line
>> > 143)
>> > LB-OPERATIONS                     0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line
>> > 143)
>> > GMFD                              0         1
>> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line
>> > 143)
>> > MESSAGE-DESERIALIZER-POOL         0         0
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,585
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/system/HintsColumnFamily-8-Data.db
>> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line
>> > 143)
>> > CONSISTENCY-MANAGER               0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line
>> > 143)
>> > LB-TARGET                         0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line
>> > 143)
>> > ROW-MUTATION-STAGE                8        49
>> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line
>> > 143)
>> > MESSAGE-STREAMING-POOL            0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line
>> > 143)
>> > LOAD-BALANCER-STAGE               0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line
>> > 143)
>> > FLUSH-SORTER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             1         1
>> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line
>> > 143)
>> > HINTED-HANDOFF-POOL               1         6
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,589
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-357-Data.db
>> >  INFO [GC inspection] 2010-08-19 16:34:03,589 GCInspector.java (line
>> > 148)
>> > CompactionManager               n/a         1
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,600
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-356-Data.db
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,603
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-359-Data.db
>> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
>> > (line 320) Compacted to
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db.
>> > 41190813/36949030 bytes for 9048 keys.  Time: 1399ms.
>> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
>> > (line 246) Compacting
>> >
>> > [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1009-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1017-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1026-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db')]
>> >  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line
>> > 116) GC
>> > for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving
>> > 8326856720
>> > used; max is 8700035072
>> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line
>> > 137)
>> > Pool Name                    Active   Pending
>> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line
>> > 143)
>> > STREAM-STAGE                      0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line
>> > 143)
>> > FILEUTILS-DELETE-POOL             0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line
>> > 143)
>> > RESPONSE-STAGE                    0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line
>> > 143)
>> > ROW-READ-STAGE                    0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line
>> > 143)
>> > LB-OPERATIONS                     0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line
>> > 143)
>> > GMFD                              0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line
>> > 143)
>> > MESSAGE-DESERIALIZER-POOL         1         1
>> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line
>> > 143)
>> > CONSISTENCY-MANAGER               0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line
>> > 143)
>> > LB-TARGET                         0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line
>> > 143)
>> > ROW-MUTATION-STAGE                4         1
>> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line
>> > 143)
>> > MESSAGE-STREAMING-POOL            0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line
>> > 143)
>> > LOAD-BALANCER-STAGE               0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line
>> > 143)
>> > FLUSH-SORTER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line
>> > 143)
>> > HINTED-HANDOFF-POOL               1         6
>> >  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line
>> > 148)
>> > CompactionManager               n/a         1
>> >  INFO [Timer-0] 2010-08-19 16:36:00,785 Gossiper.java (line 180)
>> > InetAddress
>> > /10.4.0.205 is now dead.
>> >  INFO [Timer-0] 2010-08-19 16:36:00,786 Gossiper.java (line 180)
>> > InetAddress
>> > /10.4.0.204 is now dead.
>> >  INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line
>> > 116) GC
>> > for ConcurrentMarkSweep: 37122 ms, 157488 reclaimed leaving 8342836376
>> > used;
>> > max is 8700035072
>> >  INFO [GC inspection] 2010-08-19 16:36:00,789 GCInspector.java (line
>> > 137)
>> > Pool Name                    Active   Pending
>> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line
>> > 143)
>> > STREAM-STAGE                      0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line
>> > 143)
>> > FILEUTILS-DELETE-POOL             0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line
>> > 143)
>> > RESPONSE-STAGE                    0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line
>> > 143)
>> > ROW-READ-STAGE                    0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line
>> > 143)
>> > LB-OPERATIONS                     0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line
>> > 143)
>> > GMFD                              0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line
>> > 143)
>> > MESSAGE-DESERIALIZER-POOL         0        82
>> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line
>> > 143)
>> > CONSISTENCY-MANAGER               0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line
>> > 143)
>> > LB-TARGET                         0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line
>> > 143)
>> > ROW-MUTATION-STAGE                3       770
>> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line
>> > 143)
>> > MESSAGE-STREAMING-POOL            0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line
>> > 143)
>> > LOAD-BALANCER-STAGE               0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line
>> > 143)
>> > FLUSH-SORTER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,796
>> > MessageDeserializationTask.java (line 47) dropping message (44,076ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
>> > MessageDeserializationTask.java (line 47) dropping message (44,077ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
>> > MessageDeserializationTask.java (line 47) dropping message (44,077ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,798
>> > MessageDeserializationTask.java (line 47) dropping message (44,078ms
>> > past
>> > timeout)
>> >  INFO [Timer-0] 2010-08-19 16:36:00,797 Gossiper.java (line 180)
>> > InetAddress
>> > /10.4.0.203 is now dead.
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,797
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,805
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,808
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db
>> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,812
>> > SSTableDeletingReference.java (line 104) Deleted
>> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db
>> >  INFO [GC inspection] 2010-08-19 16:36:00,833 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 0         0
>> >  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line
>> > 143)
>> > HINTED-HANDOFF-POOL               1         6
>> >  INFO [GMFD:1] 2010-08-19 16:36:00,834 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.204 is now UP
>> >  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line
>> > 148)
>> > CompactionManager               n/a         1
>> >  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.203 is now UP
>> >  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578)
>> > InetAddress
>> > /10.4.0.205 is now UP
>> >  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898
>> > ColumnFamilyStore.java
>> > (line 357) Point.Time.Data.Value has reached its threshold; switching in
>> > a
>> > fresh Memtable at
>> >
>> > CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
>> > position=41642302)
>> >  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898
>> > ColumnFamilyStore.java
>> > (line 609) Enqueuing flush of
>> > Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
>> > operations)
>> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:36:00,898 Memtable.java (line
>> > 148)
>> > Writing Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
>> > operations)
>> > ERROR [Thread-121] 2010-08-19 16:36:00,911 CassandraDaemon.java (line
>> > 83)
>> > Uncaught exception in thread Thread[Thread-121,5,main]
>> > java.lang.OutOfMemoryError: Java heap space
>> >     at
>> >
>> > org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)
>> >
>> >
>> >
>> > Messages prior to last crash.
>> >
>> >
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:06:32,245
>> > MessageDeserializationTask.java (line 47) dropping message (525,696ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,391
>> > MessageDeserializationTask.java (line 47) dropping message (562,843ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:07:09,391 GCInspector.java (line
>> > 143)
>> > LB-OPERATIONS                     0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
>> > MessageDeserializationTask.java (line 47) dropping message (562,843ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:07:09,392 GCInspector.java (line
>> > 143)
>> > GMFD                              0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
>> > MessageDeserializationTask.java (line 47) dropping message (562,844ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,394
>> > MessageDeserializationTask.java (line 47) dropping message (562,846ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,315
>> > MessageDeserializationTask.java (line 47) dropping message (595,767ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
>> > MessageDeserializationTask.java (line 47) dropping message (595,768ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
>> > MessageDeserializationTask.java (line 47) dropping message (595,768ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:07:41,965 GCInspector.java (line
>> > 143)
>> > MESSAGE-DESERIALIZER-POOL         1    567231
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,189
>> > MessageDeserializationTask.java (line 47) dropping message (631,641ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:08:18,192 GCInspector.java (line
>> > 143)
>> > CONSISTENCY-MANAGER               0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,193
>> > MessageDeserializationTask.java (line 47) dropping message (631,645ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:18,355
>> > MessageDeserializationTask.java (line 47) dropping message (691,807ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:09:18,356 GCInspector.java (line
>> > 143)
>> > LB-TARGET                         0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:44,730
>> > MessageDeserializationTask.java (line 47) dropping message (691,808ms
>> > past
>> > timeout)
>> > ERROR [COMPACTION-POOL:1] 2010-08-19 18:18:42,658 CassandraDaemon.java
>> > (line
>> > 83) Uncaught exception in thread Thread[COMPACTION-POOL:1,1,main]
>> > java.util.concurrent.ExecutionException: java.lang.OutOfMemoryError:
>> > Java
>> > heap space
>> >     at
>> > java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
>> >     at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>> >     at
>> >
>> > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:86)
>> >     at
>> >
>> > org.apache.cassandra.db.CompactionManager$CompactionExecutor.afterExecute(CompactionManager.java:577)
>> >     at
>> >
>> > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888)
>> >     at
>> >
>> > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>> >     at java.lang.Thread.run(Thread.java:619)
>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>> >     at
>> >
>> > org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:84)
>> >     at
>> >
>> > org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:335)
>> >     at
>> >
>> > org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:284)
>> >     at
>> >
>> > org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:117)
>> >     at
>> >
>> > org.apache.cassandra.db.ColumnFamilySerializer.deserializeFromSSTable(ColumnFamilySerializer.java:160)
>> >     at
>> >
>> > org.apache.cassandra.io.IteratingRow.getColumnFamily(IteratingRow.java:79)
>> >     at
>> >
>> > org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:112)
>> >     at
>> >
>> > org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:41)
>> >     at
>> >
>> > org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:73)
>> >     at
>> >
>> > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>> >     at
>> >
>> > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>> >     at
>> >
>> > org.apache.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183)
>> >     at
>> >
>> > org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94)
>> >     at
>> >
>> > org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:295)
>> >     at
>> >
>> > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:102)
>> >     at
>> >
>> > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:83)
>> >     at
>> > java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>> >     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>> >     at
>> >
>> > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>> >     ... 2 more
>> > ERROR [Thread-13] 2010-08-19 18:18:42,658 CassandraDaemon.java (line 83)
>> > Uncaught exception in thread Thread[Thread-13,5,main]
>> > java.lang.OutOfMemoryError: Java heap space
>> >     at java.util.concurrent.FutureTask.<init>(FutureTask.java:64)
>> >     at
>> >
>> > java.util.concurrent.AbstractExecutorService.newTaskFor(AbstractExecutorService.java:58)
>> >     at
>> >
>> > java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:77)
>> >     at
>> >
>> > org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:73)
>> >  INFO [GC inspection] 2010-08-19 18:18:42,658 GCInspector.java (line
>> > 143)
>> > ROW-MUTATION-STAGE                3         3
>> >  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line
>> > 143)
>> > MESSAGE-STREAMING-POOL            0         0
>> >  INFO [COMPACTION-POOL:1] 2010-08-19 18:18:42,659 CompactionManager.java
>> > (line 246) Compacting
>> >
>> > [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1071-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1072-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1073-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1074-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1075-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1076-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1077-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1078-Data.db')]
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,658
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,110ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line
>> > 143)
>> > LOAD-BALANCER-STAGE               0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,667
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,119ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
>> > past
>> > timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:18:42,960 GCInspector.java (line
>> > 143)
>> > FLUSH-SORTER-POOL                 0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:18:42,962 GCInspector.java (line
>> > 143)
>> > MEMTABLE-POST-FLUSHER             0         0
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,963
>> > MessageDeserializationTask.java (line 47) dropping message (1,256,415ms
>> > past
>> > timeout)
>> >  INFO [GC inspection] 2010-08-19 18:18:42,965 GCInspector.java (line
>> > 143)
>> > AE-SERVICE-STAGE                  0         0
>> >  INFO [GC inspection] 2010-08-19 18:18:42,968 GCInspector.java (line
>> > 143)
>> > FLUSH-WRITER-POOL                 0         0
>> >
>> >
>> >
>> >
>>
>> First,
>> I have some general questions. Are you using random partitioner? Are
>> your tokens calculated properly to spread data around the ring or did
>> cassandra auto-choose tokens? What to your system vitals look like?
>> What is the disk statistics are the disks saturated? Is the system in
>> IO wait? top, iostat,sar. What does CFSTATS say? What is your
>> largestRow? What is your average row? What is your cache hit rate?
>> What is your replication factor?  Exactly how many rows are you
>> inserting a second? If you throttle your load while tuning up your
>> settings that is more scientific then firing full blast bulk loads
>> that causes crashes.
>>
>> I notice you said this... 6 x Nodes with 8 cores, 8G heap, 24g physical.
>> I would go higher with the heap 8 cores, 8G heap, 24g physical. If you
>> are hitting OOM with 8G go higher. If you are going to use row cache
>> you should probably bring this really high.
>>
>> You are getting OOM. I notice you tried rowcaching and keycaching. How
>> did you set this to fixed size or % ? You should almost always set it
>> to fixed size.
>>
>> Here is something profound that I learned:
>> Larger row cache can SAVE you memory.
>> What huh? How can that be? more memory uses less memory?
>> Having a small row/key cache caused higher disk utilization. If all
>> requests are going to disk request take longer using more memory for
>> longer. BUT if you get that request served off Row Cache. Pow Zip Bang
>> the request is in and out fast, and you never get into using those
>> longer lived java memory pools.
>> Then again if your workload is ALL write, caches will not help you with
>> that.
>>
>> Set this low:
>>  <RowWarningThresholdInMB>10</RowWarningThresholdInMB>
>>  I found that rows with millions of columns would cause havoc for me.
>> This helped me find some bad rows.
>>
>> How much data is on each node 'nodetool ring' output
>
>

The largest row on the other nodes is as much as 800megs.

This could be your problem. I had a key with 4,000,000 columns its
size was 128 MB. I suspect any operation that touched it would RPC
timeout, and the nodes that had to work with that row would OOM and
die on me all the time. I count not do anything with it, get,  count,
key slice. This could explain why your message serialization pool is
full.

http://wiki.apache.org/cassandra/CassandraLimitations

In my case before you hit the physical limitation of 2GB rows this
large were causing me problems.

Re: Node OOM Problems

Posted by Peter Schuller <pe...@infidyne.com>.
> of a rogue large row is one I never considered. The largest row on the other
> nodes is as much as 800megs. I can not get a cfstats reading on the bad node

WIth 0.6 I can definitely see this being a problem if I understand its
behavior correctly (I have not actually used 0.6 even for testing). In
particular such amounts of data is likely to end up directly in the
old generation in the GC (normally the young generation is smaller
than 800 mb, and that does not take into account the time it takes to
actually read and process those large rows and the likelyhood of a
young-generation gc triggering anyway due to other normal activity).
Having a single value be 10% of the total heap size is likely to be
problematic in general (that could be said in some cases (e.g. 32 bit
virtual memory space and fragmentation issues) for e.g.
malloc()/free() too; algorithms solving general allocation problems
are often not very good at dealing with extreme outliers).

> so do not know how big its largest row is. I will raise memory to 16gb and
> see if that makes a difference. I had though that the java heap sizes that
> high had issues on their own in term of GC.

The garbage collector may or may not have issues in particular cases,
and to some extent the heap size is definitely a factor. However, a
lot of other things play in, including the application's overall
allocation behavior and pointer writing behavior. A large heap size in
and of itself should not be a huge problem; if you combine a very
large heap size with lots of allocation and lots of behavior that is
difficult for the particular GC to deal with, you may be more likely
to have problems.

My gut feeling with Cassandra is that I expect it to be fine, with the
worst case being having to tweak GC settings to e.g. make the
concurrent mark/sweep phases kick in earlier. In other words I would
not expect Cassandra to be an application where it becomes problematic
to keep CMS pause times down. However, I have no hard evidence of
that.

I'd be very interested to hear if people have other experiences in
production environments with very large heap sizes.

-- 
/ Peter Schuller

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
We are using the random partitioner. The tokens we defined manually and data
is almost totally equal among nodes, 15GB per node when the trouble started.
System vitals look fine. CPU load is ~500% for java, iostats are low,
everything for all practical purposes looks "normal". Cfstats reports ~1-2
ms for a read and ~.015-.02 for a write. All cache is now turned off as that
seems to cause issues a lot faster. In terms of the current load, it is
single insert statements. 6 nodes and 4 processes pointed to each node. That
is not blasting away in my book.

We can go a lot higher with memory, but with an 8g heap an NO cache thought
this was a good number. With cache we would go higher on the memory but we
are trying to get past this issue before even using cache. The possibility
of a rogue large row is one I never considered. The largest row on the other
nodes is as much as 800megs. I can not get a cfstats reading on the bad node
so do not know how big its largest row is. I will raise memory to 16gb and
see if that makes a difference. I had though that the java heap sizes that
high had issues on their own in term of GC.

Thanks.


On Thu, Aug 19, 2010 at 9:44 PM, Edward Capriolo <ed...@gmail.com>wrote:

> On Thu, Aug 19, 2010 at 2:48 PM, Wayne <wa...@gmail.com> wrote:
> > I am having some serious problems keeping a 6 node cluster up and running
> > and stable under load. Any help would be greatly appreciated.
> >
> > Basically it always comes back to OOM errors that never seem to subside.
> > After 5 minutes or 3 hours of heavy load depending on settings one or
> more
> > nodes seem to go down with an OOM error. Upon restart the node tries to
> get
> > going but ends up with an OOM error again and again and again. This
> patterns
> > repeats and repeats with no way out. Once a node goes south on me
> deleting
> > everything and starting from scratch is my only option. I can never get
> it
> > to stay up again.
> >
> > I will try to describe the latest in detail. I am running a migration
> from a
> > MySQL database with 24 processes against 6 nodes in python using the std
> > thrift client. Should be a piece of cake in cassandra but no matter what
> I
> > have tried it always seems to kill a node after not too long.
> >
> > 6 x Nodes with 8 cores, 8G heap, 24g physical
> > 0.6.4
> > JVM Options for lower compaction priority
> > JVM 1.6.0_21-b06
> >
> > After 3 hours one node hard crashed with an OOM error. See below for
> > messages.
> >
> > I restart and then see nothing but MessageDeserializationTask & some
> > GCInspector messages.
> >
> >  INFO [GC inspection] 2010-08-19 17:06:09,228 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             1         1
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,227
> > MessageDeserializationTask.java (line 47) dropping message (52,288ms past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,229
> > MessageDeserializationTask.java (line 47) dropping message (52,290ms past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 1         1
> >
> >
> > After the 2nd restart it dies/freezes completely without actually
> crashing
> > (kill -9 is required). I knew it was dead because nothing was logged for
> 30
> > minutes to the log and nodetool still registered the node as down.
> >
> > After the 3rd reboot it comes back UP but then eventually crashes again.
> See
> > below for details.
> >
> > I have tried using key or row cache but cache makes the problem happen in
> 5
> > minutes as there is less memory and the OOM happens almost immediately. I
> > have played with setting less concurrent readers and writers and now have
> it
> > set to 16/16 (read/write). I have tried to set the mmap_index_only with
> no
> > change as the issue is really a JVM issue. All other settings are
> > std/default.
> >
> > Does anyone keep their nodes up under load over time? I have been working
> > with cassandra for a while now and still have yet to keep anything up
> under
> > load for very long. I know nothing about java, and frankly am starting to
> > wonder if I need to be a java programmer to use use cassandra/run in
> debug
> > mode. Any help would be greatly appreciated. These issues in the past
> made
> > me try to use hbase which was solid as a rock (comparatively) but SLOW
> and
> > too complicated...I came back again thinking .6.4 and soon .7 would be
> > better but it almost seems worse to me in terms of stability. Frankly
> next I
> > will have to look at other alternatives because cassandra is totally
> > unstable. I do not want to give up after all this time but I am out of
> > ideas.
> >
> > HELP!
> >
> > Messages prior to first crash:
> >
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:33:19,985
> > MessageDeserializationTask.java (line 47) dropping message (12,024ms past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 16:33:19,985 GCInspector.java (line 143)
> > CONSISTENCY-MANAGER               0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
> > LB-TARGET                         0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
> > ROW-MUTATION-STAGE                1       167
> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> > MESSAGE-STREAMING-POOL            0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> > LOAD-BALANCER-STAGE               0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> > FLUSH-SORTER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
> > HINTED-HANDOFF-POOL               1         1
> >  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 148)
> > CompactionManager               n/a         0
> >  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.202 is now UP
> >  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.203 is now UP
> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.201 is now UP
> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.204 is now UP
> >  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.205 is now UP
> >  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254
> ColumnFamilyStore.java
> > (line 357) Point.Time.Data.Value has reached its threshold; switching in
> a
> > fresh Memtable at
> >
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> > position=15371920)
> >  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254
> ColumnFamilyStore.java
> > (line 609) Enqueuing flush of
> > Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
> operations)
> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:21,254 Memtable.java (line
> 148)
> > Writing Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
> > operations)
> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:22,090 Memtable.java (line
> 162)
> > Completed flushing
> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-362-Data.db
> >  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289
> ColumnFamilyStore.java
> > (line 357) Point.Data.Time.Value has reached its threshold; switching in
> a
> > fresh Memtable at
> >
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> > position=25251401)
> >  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289
> ColumnFamilyStore.java
> > (line 609) Enqueuing flush of
> > Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
> operations)
> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,289 Memtable.java (line
> 148)
> > Writing Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
> > operations)
> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,785 Memtable.java (line
> 162)
> > Completed flushing
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:33:23,787 CompactionManager.java
> > (line 246) Compacting
> >
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db')]
> >  INFO [GC inspection] 2010-08-19 16:34:03,565 GCInspector.java (line 116)
> GC
> > for ConcurrentMarkSweep: 39651 ms, 414341376 reclaimed leaving 8139886200
> > used; max is 8700035072
> >  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 137)
> > Pool Name                    Active   Pending
> >  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 143)
> > STREAM-STAGE                      0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
> > FILEUTILS-DELETE-POOL             0         0
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,567
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-358-Data.db
> >  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
> > RESPONSE-STAGE                    0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
> > ROW-READ-STAGE                    0         0
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,584
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/system/HintsColumnFamily-7-Data.db
> >  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
> > LB-OPERATIONS                     0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> > GMFD                              0         1
> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> > MESSAGE-DESERIALIZER-POOL         0         0
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,585
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/system/HintsColumnFamily-8-Data.db
> >  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> > CONSISTENCY-MANAGER               0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
> > LB-TARGET                         0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
> > ROW-MUTATION-STAGE                8        49
> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> > MESSAGE-STREAMING-POOL            0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> > LOAD-BALANCER-STAGE               0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> > FLUSH-SORTER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             1         1
> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> > HINTED-HANDOFF-POOL               1         6
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,589
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-357-Data.db
> >  INFO [GC inspection] 2010-08-19 16:34:03,589 GCInspector.java (line 148)
> > CompactionManager               n/a         1
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,600
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-356-Data.db
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,603
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-359-Data.db
> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
> > (line 320) Compacted to
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db.
> > 41190813/36949030 bytes for 9048 keys.  Time: 1399ms.
> >  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
> > (line 246) Compacting
> >
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1009-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1017-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1026-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db')]
> >  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line 116)
> GC
> > for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving 8326856720
> > used; max is 8700035072
> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 137)
> > Pool Name                    Active   Pending
> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
> > STREAM-STAGE                      0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
> > FILEUTILS-DELETE-POOL             0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> > RESPONSE-STAGE                    0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> > ROW-READ-STAGE                    0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> > LB-OPERATIONS                     0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> > GMFD                              0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> > MESSAGE-DESERIALIZER-POOL         1         1
> >  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> > CONSISTENCY-MANAGER               0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> > LB-TARGET                         0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> > ROW-MUTATION-STAGE                4         1
> >  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> > MESSAGE-STREAMING-POOL            0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> > LOAD-BALANCER-STAGE               0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> > FLUSH-SORTER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 143)
> > HINTED-HANDOFF-POOL               1         6
> >  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 148)
> > CompactionManager               n/a         1
> >  INFO [Timer-0] 2010-08-19 16:36:00,785 Gossiper.java (line 180)
> InetAddress
> > /10.4.0.205 is now dead.
> >  INFO [Timer-0] 2010-08-19 16:36:00,786 Gossiper.java (line 180)
> InetAddress
> > /10.4.0.204 is now dead.
> >  INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116)
> GC
> > for ConcurrentMarkSweep: 37122 ms, 157488 reclaimed leaving 8342836376
> used;
> > max is 8700035072
> >  INFO [GC inspection] 2010-08-19 16:36:00,789 GCInspector.java (line 137)
> > Pool Name                    Active   Pending
> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> > STREAM-STAGE                      0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> > FILEUTILS-DELETE-POOL             0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> > RESPONSE-STAGE                    0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> > ROW-READ-STAGE                    0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> > LB-OPERATIONS                     0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> > GMFD                              0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> > MESSAGE-DESERIALIZER-POOL         0        82
> >  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> > CONSISTENCY-MANAGER               0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> > LB-TARGET                         0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> > ROW-MUTATION-STAGE                3       770
> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> > MESSAGE-STREAMING-POOL            0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> > LOAD-BALANCER-STAGE               0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> > FLUSH-SORTER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,796
> > MessageDeserializationTask.java (line 47) dropping message (44,076ms past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
> > MessageDeserializationTask.java (line 47) dropping message (44,077ms past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
> > MessageDeserializationTask.java (line 47) dropping message (44,077ms past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,798
> > MessageDeserializationTask.java (line 47) dropping message (44,078ms past
> > timeout)
> >  INFO [Timer-0] 2010-08-19 16:36:00,797 Gossiper.java (line 180)
> InetAddress
> > /10.4.0.203 is now dead.
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,797
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,805
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,808
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db
> >  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,812
> > SSTableDeletingReference.java (line 104) Deleted
> > /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db
> >  INFO [GC inspection] 2010-08-19 16:36:00,833 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 0         0
> >  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 143)
> > HINTED-HANDOFF-POOL               1         6
> >  INFO [GMFD:1] 2010-08-19 16:36:00,834 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.204 is now UP
> >  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 148)
> > CompactionManager               n/a         1
> >  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.203 is now UP
> >  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578)
> InetAddress
> > /10.4.0.205 is now UP
> >  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898
> ColumnFamilyStore.java
> > (line 357) Point.Time.Data.Value has reached its threshold; switching in
> a
> > fresh Memtable at
> >
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> > position=41642302)
> >  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898
> ColumnFamilyStore.java
> > (line 609) Enqueuing flush of
> > Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
> operations)
> >  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:36:00,898 Memtable.java (line
> 148)
> > Writing Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
> > operations)
> > ERROR [Thread-121] 2010-08-19 16:36:00,911 CassandraDaemon.java (line 83)
> > Uncaught exception in thread Thread[Thread-121,5,main]
> > java.lang.OutOfMemoryError: Java heap space
> >     at
> >
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)
> >
> >
> >
> > Messages prior to last crash.
> >
> >
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:06:32,245
> > MessageDeserializationTask.java (line 47) dropping message (525,696ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,391
> > MessageDeserializationTask.java (line 47) dropping message (562,843ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:07:09,391 GCInspector.java (line 143)
> > LB-OPERATIONS                     0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
> > MessageDeserializationTask.java (line 47) dropping message (562,843ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:07:09,392 GCInspector.java (line 143)
> > GMFD                              0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
> > MessageDeserializationTask.java (line 47) dropping message (562,844ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,394
> > MessageDeserializationTask.java (line 47) dropping message (562,846ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,315
> > MessageDeserializationTask.java (line 47) dropping message (595,767ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
> > MessageDeserializationTask.java (line 47) dropping message (595,768ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
> > MessageDeserializationTask.java (line 47) dropping message (595,768ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:07:41,965 GCInspector.java (line 143)
> > MESSAGE-DESERIALIZER-POOL         1    567231
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,189
> > MessageDeserializationTask.java (line 47) dropping message (631,641ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:08:18,192 GCInspector.java (line 143)
> > CONSISTENCY-MANAGER               0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,193
> > MessageDeserializationTask.java (line 47) dropping message (631,645ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:18,355
> > MessageDeserializationTask.java (line 47) dropping message (691,807ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:09:18,356 GCInspector.java (line 143)
> > LB-TARGET                         0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:44,730
> > MessageDeserializationTask.java (line 47) dropping message (691,808ms
> past
> > timeout)
> > ERROR [COMPACTION-POOL:1] 2010-08-19 18:18:42,658 CassandraDaemon.java
> (line
> > 83) Uncaught exception in thread Thread[COMPACTION-POOL:1,1,main]
> > java.util.concurrent.ExecutionException: java.lang.OutOfMemoryError: Java
> > heap space
> >     at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
> >     at java.util.concurrent.FutureTask.get(FutureTask.java:83)
> >     at
> >
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:86)
> >     at
> >
> org.apache.cassandra.db.CompactionManager$CompactionExecutor.afterExecute(CompactionManager.java:577)
> >     at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888)
> >     at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> >     at java.lang.Thread.run(Thread.java:619)
> > Caused by: java.lang.OutOfMemoryError: Java heap space
> >     at
> >
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:84)
> >     at
> >
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:335)
> >     at
> >
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:284)
> >     at
> >
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:117)
> >     at
> >
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeFromSSTable(ColumnFamilySerializer.java:160)
> >     at
> >
> org.apache.cassandra.io.IteratingRow.getColumnFamily(IteratingRow.java:79)
> >     at
> >
> org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:112)
> >     at
> >
> org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:41)
> >     at
> >
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:73)
> >     at
> >
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
> >     at
> >
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
> >     at
> >
> org.apache.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183)
> >     at
> >
> org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94)
> >     at
> >
> org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:295)
> >     at
> >
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:102)
> >     at
> >
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:83)
> >     at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> >     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> >     at
> >
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> >     ... 2 more
> > ERROR [Thread-13] 2010-08-19 18:18:42,658 CassandraDaemon.java (line 83)
> > Uncaught exception in thread Thread[Thread-13,5,main]
> > java.lang.OutOfMemoryError: Java heap space
> >     at java.util.concurrent.FutureTask.<init>(FutureTask.java:64)
> >     at
> >
> java.util.concurrent.AbstractExecutorService.newTaskFor(AbstractExecutorService.java:58)
> >     at
> >
> java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:77)
> >     at
> >
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:73)
> >  INFO [GC inspection] 2010-08-19 18:18:42,658 GCInspector.java (line 143)
> > ROW-MUTATION-STAGE                3         3
> >  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
> > MESSAGE-STREAMING-POOL            0         0
> >  INFO [COMPACTION-POOL:1] 2010-08-19 18:18:42,659 CompactionManager.java
> > (line 246) Compacting
> >
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1071-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1072-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1073-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1074-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1075-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1076-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1077-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1078-Data.db')]
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,658
> > MessageDeserializationTask.java (line 47) dropping message (1,256,110ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
> > LOAD-BALANCER-STAGE               0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,667
> > MessageDeserializationTask.java (line 47) dropping message (1,256,119ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:18:42,960 GCInspector.java (line 143)
> > FLUSH-SORTER-POOL                 0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> > MessageDeserializationTask.java (line 47) dropping message (1,256,412ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:18:42,962 GCInspector.java (line 143)
> > MEMTABLE-POST-FLUSHER             0         0
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,963
> > MessageDeserializationTask.java (line 47) dropping message (1,256,415ms
> past
> > timeout)
> >  INFO [GC inspection] 2010-08-19 18:18:42,965 GCInspector.java (line 143)
> > AE-SERVICE-STAGE                  0         0
> >  INFO [GC inspection] 2010-08-19 18:18:42,968 GCInspector.java (line 143)
> > FLUSH-WRITER-POOL                 0         0
> >
> >
> >
> >
>
> First,
> I have some general questions. Are you using random partitioner? Are
> your tokens calculated properly to spread data around the ring or did
> cassandra auto-choose tokens? What to your system vitals look like?
> What is the disk statistics are the disks saturated? Is the system in
> IO wait? top, iostat,sar. What does CFSTATS say? What is your
> largestRow? What is your average row? What is your cache hit rate?
> What is your replication factor?  Exactly how many rows are you
> inserting a second? If you throttle your load while tuning up your
> settings that is more scientific then firing full blast bulk loads
> that causes crashes.
>
> I notice you said this... 6 x Nodes with 8 cores, 8G heap, 24g physical.
> I would go higher with the heap 8 cores, 8G heap, 24g physical. If you
> are hitting OOM with 8G go higher. If you are going to use row cache
> you should probably bring this really high.
>
> You are getting OOM. I notice you tried rowcaching and keycaching. How
> did you set this to fixed size or % ? You should almost always set it
> to fixed size.
>
> Here is something profound that I learned:
> Larger row cache can SAVE you memory.
> What huh? How can that be? more memory uses less memory?
> Having a small row/key cache caused higher disk utilization. If all
> requests are going to disk request take longer using more memory for
> longer. BUT if you get that request served off Row Cache. Pow Zip Bang
> the request is in and out fast, and you never get into using those
> longer lived java memory pools.
> Then again if your workload is ALL write, caches will not help you with
> that.
>
> Set this low:
>  <RowWarningThresholdInMB>10</RowWarningThresholdInMB>
>  I found that rows with millions of columns would cause havoc for me.
> This helped me find some bad rows.
>
> How much data is on each node 'nodetool ring' output
>

Re: Node OOM Problems

Posted by Edward Capriolo <ed...@gmail.com>.
On Thu, Aug 19, 2010 at 2:48 PM, Wayne <wa...@gmail.com> wrote:
> I am having some serious problems keeping a 6 node cluster up and running
> and stable under load. Any help would be greatly appreciated.
>
> Basically it always comes back to OOM errors that never seem to subside.
> After 5 minutes or 3 hours of heavy load depending on settings one or more
> nodes seem to go down with an OOM error. Upon restart the node tries to get
> going but ends up with an OOM error again and again and again. This patterns
> repeats and repeats with no way out. Once a node goes south on me deleting
> everything and starting from scratch is my only option. I can never get it
> to stay up again.
>
> I will try to describe the latest in detail. I am running a migration from a
> MySQL database with 24 processes against 6 nodes in python using the std
> thrift client. Should be a piece of cake in cassandra but no matter what I
> have tried it always seems to kill a node after not too long.
>
> 6 x Nodes with 8 cores, 8G heap, 24g physical
> 0.6.4
> JVM Options for lower compaction priority
> JVM 1.6.0_21-b06
>
> After 3 hours one node hard crashed with an OOM error. See below for
> messages.
>
> I restart and then see nothing but MessageDeserializationTask & some
> GCInspector messages.
>
>  INFO [GC inspection] 2010-08-19 17:06:09,228 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             1         1
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,227
> MessageDeserializationTask.java (line 47) dropping message (52,288ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 17:06:09,229
> MessageDeserializationTask.java (line 47) dropping message (52,290ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 17:06:09,229 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 1         1
>
>
> After the 2nd restart it dies/freezes completely without actually crashing
> (kill -9 is required). I knew it was dead because nothing was logged for 30
> minutes to the log and nodetool still registered the node as down.
>
> After the 3rd reboot it comes back UP but then eventually crashes again. See
> below for details.
>
> I have tried using key or row cache but cache makes the problem happen in 5
> minutes as there is less memory and the OOM happens almost immediately. I
> have played with setting less concurrent readers and writers and now have it
> set to 16/16 (read/write). I have tried to set the mmap_index_only with no
> change as the issue is really a JVM issue. All other settings are
> std/default.
>
> Does anyone keep their nodes up under load over time? I have been working
> with cassandra for a while now and still have yet to keep anything up under
> load for very long. I know nothing about java, and frankly am starting to
> wonder if I need to be a java programmer to use use cassandra/run in debug
> mode. Any help would be greatly appreciated. These issues in the past made
> me try to use hbase which was solid as a rock (comparatively) but SLOW and
> too complicated...I came back again thinking .6.4 and soon .7 would be
> better but it almost seems worse to me in terms of stability. Frankly next I
> will have to look at other alternatives because cassandra is totally
> unstable. I do not want to give up after all this time but I am out of
> ideas.
>
> HELP!
>
> Messages prior to first crash:
>
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:33:19,985
> MessageDeserializationTask.java (line 47) dropping message (12,024ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 16:33:19,985 GCInspector.java (line 143)
> CONSISTENCY-MANAGER               0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
> LB-TARGET                         0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,986 GCInspector.java (line 143)
> ROW-MUTATION-STAGE                1       167
>  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> MESSAGE-STREAMING-POOL            0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> LOAD-BALANCER-STAGE               0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> FLUSH-SORTER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,987 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 143)
> HINTED-HANDOFF-POOL               1         1
>  INFO [GC inspection] 2010-08-19 16:33:19,988 GCInspector.java (line 148)
> CompactionManager               n/a         0
>  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578) InetAddress
> /10.4.0.202 is now UP
>  INFO [GMFD:1] 2010-08-19 16:33:19,989 Gossiper.java (line 578) InetAddress
> /10.4.0.203 is now UP
>  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
> /10.4.0.201 is now UP
>  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
> /10.4.0.204 is now UP
>  INFO [GMFD:1] 2010-08-19 16:33:19,990 Gossiper.java (line 578) InetAddress
> /10.4.0.205 is now UP
>  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254 ColumnFamilyStore.java
> (line 357) Point.Time.Data.Value has reached its threshold; switching in a
> fresh Memtable at
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> position=15371920)
>  INFO [ROW-MUTATION-STAGE:19] 2010-08-19 16:33:21,254 ColumnFamilyStore.java
> (line 609) Enqueuing flush of
> Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574 operations)
>  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:21,254 Memtable.java (line 148)
> Writing Memtable-Point.Time.Data.Value@951707146(3462840 bytes, 314574
> operations)
>  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:22,090 Memtable.java (line 162)
> Completed flushing
> /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-362-Data.db
>  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289 ColumnFamilyStore.java
> (line 357) Point.Data.Time.Value has reached its threshold; switching in a
> fresh Memtable at
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> position=25251401)
>  INFO [ROW-MUTATION-STAGE:58] 2010-08-19 16:33:23,289 ColumnFamilyStore.java
> (line 609) Enqueuing flush of
> Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746 operations)
>  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,289 Memtable.java (line 148)
> Writing Memtable-Point.Data.Time.Value@142023252(6330574 bytes, 314746
> operations)
>  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:33:23,785 Memtable.java (line 162)
> Completed flushing
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
>  INFO [COMPACTION-POOL:1] 2010-08-19 16:33:23,787 CompactionManager.java
> (line 246) Compacting
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db')]
>  INFO [GC inspection] 2010-08-19 16:34:03,565 GCInspector.java (line 116) GC
> for ConcurrentMarkSweep: 39651 ms, 414341376 reclaimed leaving 8139886200
> used; max is 8700035072
>  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 137)
> Pool Name                    Active   Pending
>  INFO [GC inspection] 2010-08-19 16:34:03,566 GCInspector.java (line 143)
> STREAM-STAGE                      0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
> FILEUTILS-DELETE-POOL             0         0
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,567
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-358-Data.db
>  INFO [GC inspection] 2010-08-19 16:34:03,567 GCInspector.java (line 143)
> RESPONSE-STAGE                    0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
> ROW-READ-STAGE                    0         0
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,584
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/system/HintsColumnFamily-7-Data.db
>  INFO [GC inspection] 2010-08-19 16:34:03,584 GCInspector.java (line 143)
> LB-OPERATIONS                     0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> GMFD                              0         1
>  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> MESSAGE-DESERIALIZER-POOL         0         0
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,585
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/system/HintsColumnFamily-8-Data.db
>  INFO [GC inspection] 2010-08-19 16:34:03,585 GCInspector.java (line 143)
> CONSISTENCY-MANAGER               0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
> LB-TARGET                         0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,586 GCInspector.java (line 143)
> ROW-MUTATION-STAGE                8        49
>  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> MESSAGE-STREAMING-POOL            0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> LOAD-BALANCER-STAGE               0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,587 GCInspector.java (line 143)
> FLUSH-SORTER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             1         1
>  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:34:03,588 GCInspector.java (line 143)
> HINTED-HANDOFF-POOL               1         6
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,589
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-357-Data.db
>  INFO [GC inspection] 2010-08-19 16:34:03,589 GCInspector.java (line 148)
> CompactionManager               n/a         1
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,600
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-356-Data.db
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:34:03,603
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Time.Data.Value-359-Data.db
>  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
> (line 320) Compacted to
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db.
> 41190813/36949030 bytes for 9048 keys.  Time: 1399ms.
>  INFO [COMPACTION-POOL:1] 2010-08-19 16:34:04,965 CompactionManager.java
> (line 246) Compacting
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1009-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1017-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1026-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1031-Data.db')]
>  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line 116) GC
> for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving 8326856720
> used; max is 8700035072
>  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 137)
> Pool Name                    Active   Pending
>  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
> STREAM-STAGE                      0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,657 GCInspector.java (line 143)
> FILEUTILS-DELETE-POOL             0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> RESPONSE-STAGE                    0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> ROW-READ-STAGE                    0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,658 GCInspector.java (line 143)
> LB-OPERATIONS                     0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> GMFD                              0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> MESSAGE-DESERIALIZER-POOL         1         1
>  INFO [GC inspection] 2010-08-19 16:34:46,659 GCInspector.java (line 143)
> CONSISTENCY-MANAGER               0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> LB-TARGET                         0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> ROW-MUTATION-STAGE                4         1
>  INFO [GC inspection] 2010-08-19 16:34:46,660 GCInspector.java (line 143)
> MESSAGE-STREAMING-POOL            0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> LOAD-BALANCER-STAGE               0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> FLUSH-SORTER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,661 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,662 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 143)
> HINTED-HANDOFF-POOL               1         6
>  INFO [GC inspection] 2010-08-19 16:34:46,663 GCInspector.java (line 148)
> CompactionManager               n/a         1
>  INFO [Timer-0] 2010-08-19 16:36:00,785 Gossiper.java (line 180) InetAddress
> /10.4.0.205 is now dead.
>  INFO [Timer-0] 2010-08-19 16:36:00,786 Gossiper.java (line 180) InetAddress
> /10.4.0.204 is now dead.
>  INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116) GC
> for ConcurrentMarkSweep: 37122 ms, 157488 reclaimed leaving 8342836376 used;
> max is 8700035072
>  INFO [GC inspection] 2010-08-19 16:36:00,789 GCInspector.java (line 137)
> Pool Name                    Active   Pending
>  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> STREAM-STAGE                      0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> FILEUTILS-DELETE-POOL             0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> RESPONSE-STAGE                    0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,790 GCInspector.java (line 143)
> ROW-READ-STAGE                    0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> LB-OPERATIONS                     0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> GMFD                              0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> MESSAGE-DESERIALIZER-POOL         0        82
>  INFO [GC inspection] 2010-08-19 16:36:00,791 GCInspector.java (line 143)
> CONSISTENCY-MANAGER               0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> LB-TARGET                         0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> ROW-MUTATION-STAGE                3       770
>  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> MESSAGE-STREAMING-POOL            0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,792 GCInspector.java (line 143)
> LOAD-BALANCER-STAGE               0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> FLUSH-SORTER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,793 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,796
> MessageDeserializationTask.java (line 47) dropping message (44,076ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
> MessageDeserializationTask.java (line 47) dropping message (44,077ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,797
> MessageDeserializationTask.java (line 47) dropping message (44,077ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 16:36:00,798
> MessageDeserializationTask.java (line 47) dropping message (44,078ms past
> timeout)
>  INFO [Timer-0] 2010-08-19 16:36:00,797 Gossiper.java (line 180) InetAddress
> /10.4.0.203 is now dead.
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,797
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1030-Data.db
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,805
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1028-Data.db
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,808
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1027-Data.db
>  INFO [SSTABLE-CLEANUP-TIMER] 2010-08-19 16:36:00,812
> SSTableDeletingReference.java (line 104) Deleted
> /mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1029-Data.db
>  INFO [GC inspection] 2010-08-19 16:36:00,833 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 0         0
>  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 143)
> HINTED-HANDOFF-POOL               1         6
>  INFO [GMFD:1] 2010-08-19 16:36:00,834 Gossiper.java (line 578) InetAddress
> /10.4.0.204 is now UP
>  INFO [GC inspection] 2010-08-19 16:36:00,834 GCInspector.java (line 148)
> CompactionManager               n/a         1
>  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578) InetAddress
> /10.4.0.203 is now UP
>  INFO [GMFD:1] 2010-08-19 16:36:00,839 Gossiper.java (line 578) InetAddress
> /10.4.0.205 is now UP
>  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898 ColumnFamilyStore.java
> (line 357) Point.Time.Data.Value has reached its threshold; switching in a
> fresh Memtable at
> CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1282235556898.log',
> position=41642302)
>  INFO [ROW-MUTATION-STAGE:34] 2010-08-19 16:36:00,898 ColumnFamilyStore.java
> (line 609) Enqueuing flush of
> Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579 operations)
>  INFO [FLUSH-WRITER-POOL:1] 2010-08-19 16:36:00,898 Memtable.java (line 148)
> Writing Memtable-Point.Time.Data.Value@543707511(4032380 bytes, 374579
> operations)
> ERROR [Thread-121] 2010-08-19 16:36:00,911 CassandraDaemon.java (line 83)
> Uncaught exception in thread Thread[Thread-121,5,main]
> java.lang.OutOfMemoryError: Java heap space
>     at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)
>
>
>
> Messages prior to last crash.
>
>
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:06:32,245
> MessageDeserializationTask.java (line 47) dropping message (525,696ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,391
> MessageDeserializationTask.java (line 47) dropping message (562,843ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:07:09,391 GCInspector.java (line 143)
> LB-OPERATIONS                     0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
> MessageDeserializationTask.java (line 47) dropping message (562,843ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:07:09,392 GCInspector.java (line 143)
> GMFD                              0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,392
> MessageDeserializationTask.java (line 47) dropping message (562,844ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:09,394
> MessageDeserializationTask.java (line 47) dropping message (562,846ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,315
> MessageDeserializationTask.java (line 47) dropping message (595,767ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
> MessageDeserializationTask.java (line 47) dropping message (595,768ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:07:42,316
> MessageDeserializationTask.java (line 47) dropping message (595,768ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:07:41,965 GCInspector.java (line 143)
> MESSAGE-DESERIALIZER-POOL         1    567231
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,189
> MessageDeserializationTask.java (line 47) dropping message (631,641ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:08:18,192 GCInspector.java (line 143)
> CONSISTENCY-MANAGER               0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:08:18,193
> MessageDeserializationTask.java (line 47) dropping message (631,645ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:18,355
> MessageDeserializationTask.java (line 47) dropping message (691,807ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:09:18,356 GCInspector.java (line 143)
> LB-TARGET                         0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:09:44,730
> MessageDeserializationTask.java (line 47) dropping message (691,808ms past
> timeout)
> ERROR [COMPACTION-POOL:1] 2010-08-19 18:18:42,658 CassandraDaemon.java (line
> 83) Uncaught exception in thread Thread[COMPACTION-POOL:1,1,main]
> java.util.concurrent.ExecutionException: java.lang.OutOfMemoryError: Java
> heap space
>     at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
>     at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>     at
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:86)
>     at
> org.apache.cassandra.db.CompactionManager$CompactionExecutor.afterExecute(CompactionManager.java:577)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>     at java.lang.Thread.run(Thread.java:619)
> Caused by: java.lang.OutOfMemoryError: Java heap space
>     at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:84)
>     at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:335)
>     at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:284)
>     at
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:117)
>     at
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeFromSSTable(ColumnFamilySerializer.java:160)
>     at
> org.apache.cassandra.io.IteratingRow.getColumnFamily(IteratingRow.java:79)
>     at
> org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:112)
>     at
> org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:41)
>     at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:73)
>     at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>     at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>     at
> org.apache.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183)
>     at
> org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94)
>     at
> org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:295)
>     at
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:102)
>     at
> org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:83)
>     at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>     ... 2 more
> ERROR [Thread-13] 2010-08-19 18:18:42,658 CassandraDaemon.java (line 83)
> Uncaught exception in thread Thread[Thread-13,5,main]
> java.lang.OutOfMemoryError: Java heap space
>     at java.util.concurrent.FutureTask.<init>(FutureTask.java:64)
>     at
> java.util.concurrent.AbstractExecutorService.newTaskFor(AbstractExecutorService.java:58)
>     at
> java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:77)
>     at
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:73)
>  INFO [GC inspection] 2010-08-19 18:18:42,658 GCInspector.java (line 143)
> ROW-MUTATION-STAGE                3         3
>  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
> MESSAGE-STREAMING-POOL            0         0
>  INFO [COMPACTION-POOL:1] 2010-08-19 18:18:42,659 CompactionManager.java
> (line 246) Compacting
> [org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1071-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1072-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1073-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1074-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1075-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1076-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1077-Data.db'),org.apache.cassandra.io.SSTableReader(path='/mainraid/cassandra/data/Instance.4/Point.Data.Time.Value-1078-Data.db')]
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,658
> MessageDeserializationTask.java (line 47) dropping message (1,256,110ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:18:42,666 GCInspector.java (line 143)
> LOAD-BALANCER-STAGE               0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,667
> MessageDeserializationTask.java (line 47) dropping message (1,256,119ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:18:42,960 GCInspector.java (line 143)
> FLUSH-SORTER-POOL                 0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,960
> MessageDeserializationTask.java (line 47) dropping message (1,256,412ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:18:42,962 GCInspector.java (line 143)
> MEMTABLE-POST-FLUSHER             0         0
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-19 18:18:42,963
> MessageDeserializationTask.java (line 47) dropping message (1,256,415ms past
> timeout)
>  INFO [GC inspection] 2010-08-19 18:18:42,965 GCInspector.java (line 143)
> AE-SERVICE-STAGE                  0         0
>  INFO [GC inspection] 2010-08-19 18:18:42,968 GCInspector.java (line 143)
> FLUSH-WRITER-POOL                 0         0
>
>
>
>

First,
I have some general questions. Are you using random partitioner? Are
your tokens calculated properly to spread data around the ring or did
cassandra auto-choose tokens? What to your system vitals look like?
What is the disk statistics are the disks saturated? Is the system in
IO wait? top, iostat,sar. What does CFSTATS say? What is your
largestRow? What is your average row? What is your cache hit rate?
What is your replication factor?  Exactly how many rows are you
inserting a second? If you throttle your load while tuning up your
settings that is more scientific then firing full blast bulk loads
that causes crashes.

I notice you said this... 6 x Nodes with 8 cores, 8G heap, 24g physical.
I would go higher with the heap 8 cores, 8G heap, 24g physical. If you
are hitting OOM with 8G go higher. If you are going to use row cache
you should probably bring this really high.

You are getting OOM. I notice you tried rowcaching and keycaching. How
did you set this to fixed size or % ? You should almost always set it
to fixed size.

Here is something profound that I learned:
Larger row cache can SAVE you memory.
What huh? How can that be? more memory uses less memory?
Having a small row/key cache caused higher disk utilization. If all
requests are going to disk request take longer using more memory for
longer. BUT if you get that request served off Row Cache. Pow Zip Bang
the request is in and out fast, and you never get into using those
longer lived java memory pools.
Then again if your workload is ALL write, caches will not help you with that.

Set this low:
 <RowWarningThresholdInMB>10</RowWarningThresholdInMB>
 I found that rows with millions of columns would cause havoc for me.
This helped me find some bad rows.

How much data is on each node 'nodetool ring' output

Re: Node OOM Problems

Posted by Edward Capriolo <ed...@gmail.com>.
On Thu, Aug 19, 2010 at 4:49 PM, Wayne <wa...@gmail.com> wrote:
> What is my "live set"? Is the system CPU bound given the few statements
> below? This is from running 4 concurrent processes against the node...do I
> need to throttle back the concurrent read/writers?
>
> I do all reads/writes as Quorum. (Replication factor of 3).
>
> The memtable threshold is the default of 256.
>
> All caching is turned off.
>
> The database is pretty small, maybe a few million keys (2-3) in 4 CFs. The
> key size is pretty small. Some of the rows are pretty fat though (fatter
> than I thought). I am saving secondary indexes in separate CFs and those are
> the large rows that I think might be part of the problem. I will restart
> testing turning these off and see if I see any difference.
>
> Would an extra fat row explain repeated OOM crashes in a row? I have finally
> got the system to stabilize relatively and I even ran compaction on the bad
> node without a problem (still no row size stats).
>
> I now have several other nodes flapping with the following single error in
> the cassandra.log
> Error: Exception thrown by the agent : java.lang.NullPointerException
>
> I assume this is an unrelated problem?
>
> Thanks for all of your help!
>
> On Thu, Aug 19, 2010 at 10:26 PM, Peter Schuller
> <pe...@infidyne.com> wrote:
>>
>> So, these:
>>
>> >  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line
>> > 116) GC
>> > for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving
>> > 8326856720
>> > used; max is 8700035072
>> [snip]
>> > INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116)
>> > GC for ConcurrentMarkSweep: 37122 ms, 157488
>> > reclaimed leaving 8342836376 used; max is 8700035072
>>
>> ...show that you're live set is indeed very close to heap maximum, and
>> so concurrent mark/sweep phases run often freeing very little memory.
>> In addition the fact that it seems to take 35-45 seconds to do the
>> concurrent mark/sweep on an 8 gig heap on a modern system suggests
>> that you are probably CPU bound in cassandra at the time (meaning GC
>> is slower).
>>
>> In short you're using too much memory in comparison to the maximum
>> heap size. The expected result is to either get an OOM, or just become
>> too slow due to excessive GC activity (usually the latter followed by
>> the former).
>>
>> Now, the question is what memory is used *for*, and why. First off, to
>> get that out of the way, are you inserting with consistency level
>> ZERO? I am not sure whether it applies to 0.6.4 or not but there used
>> to be an issue involving writes at consistency level ZERO not being
>> throttled at all, meaning that if you threw writes at the system
>> faster than it would handle them, you would accumulate memory use. I
>> don't believe this is a problem with CL.ONE and above, even in 0.6.4
>> (but someone correct me if I'm wrong).
>>
>> (As an aside: I'm not sure whether the behavior was such that it might
>> explain OOM on restart as a result of accumulated commitlogs that get
>> replayed faster than memtable flushing happens. Perhaps not, not
>> sure.)
>>
>> In any case, the most important factors are what you're actually doing
>> with the cluster, but you don't say much about the data. In particular
>> how many rows and colums you're populating it with.
>>
>> The primary users of large amounts of memory in cassandra include
>> (hopefully I'm not missing something major);
>>
>> * bloom filters that are used to efficiently avoid doing I/O on
>> sstables that do not contain relevant data. the size of bloom filters
>> scale linearly with the number of row keys (not columns right? I don't
>> remember). so here we have an expected permanent, but low, memory use
>> as a result of a large database. how large is your database? 100
>> million keys? 1 billion? 10 billion?
>>
>> * the memtables; the currently active memtable and any memtables
>> currently undergoing flushing. the size of these are directly
>> controllable in the configuration file. make sure they are reasonable.
>> (If you're not sure at all, with an 8 gig heap I'd say <= 512 mb is a
>> reasonable recommendation unless you have a reason to make them
>> larger)
>>
>> * row cache and key cache, both controllable in the configuration. in
>> particular the row cache can be huge if you have configured it as
>> such.
>>
>> * to some extent unflushed commitlogs; the commit log rotation
>> threshold controls this. the default value is low enough that it
>> should not be your culprit
>>
>> So the question is what you're usage is like. How many unique rows do
>> you have? How many columns? The data size in and of itself should not
>> matter much to memory use, except of course that extremely large
>> individual values will be relevant to transient high memory use when
>> they are read/written.
>>
>> In general, lacking large row caches and such things, you should be
>> able to have hundreds of millions of entries on an 8 gb heap, assuming
>> reasonably sized keys.
>>
>> --
>> / Peter Schuller
>
>

"live set" is active data. For example I may have 900GB of data on
disk, but at given time X 10GB are in being read/written or
replicated. My "live set" would be the 10 GB

Would an extra fat row explain repeated OOM crashes in a row?

Highly likely. I had a row that was 112 MB and 4,000,000+ columns. It
caused havoc for me. Read what Peter described above. It may not be
the problem but that is a place to start looking.

Re: Node OOM Problems

Posted by Jonathan Ellis <jb...@gmail.com>.
these warnings mean you have more requests queued up than you are able
to handle.  that request queue is what is using up most of your heap
memory.

On Fri, Aug 20, 2010 at 12:17 PM, Wayne <wa...@gmail.com> wrote:
> I turned off the creation of the secondary indexes which had the large rows
> and all seemed good. Thank you for the help. I was getting
> 60k+/writes/second on the 6 node cluster.
>
> Unfortunately again three hours later a node went down. I can not even look
> at the logs when it started since they are gone/recycled due to millions of
> message deserialization messages. What are these? The node had 12,098,067
> pending message-deserializer-pool entries in tpstats. The node was up
> according to some nodes and down according to others which made it flapping
> and still trying to take requests. What is the log warning message
> deserialization task dropped message? Why would a node have 12 million of
> these?
>
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
> timeout)
>
> I do not think this is a large row problem any more. All nodes show a max
> row size around 8-9 megs.
>
> I looked at the munin charts and the disk IO seems to have spiked along with
> compaction. Could compaction kicking in cause this? I have added the 3 JVM
> settings to make compaction a lower priority. Did this help cause this to
> happen by slowing down and building up compaction on a heavily loaded
> system?
>
> Thanks in advance for any help someone can provide.
>
>
> On Fri, Aug 20, 2010 at 8:34 AM, Wayne <wa...@gmail.com> wrote:
>>
>> The NullPointerException does not crash the node. It only makes it flap/go
>> down a for short period and then it comes back up. I do not see anything
>> abnormal in the system log, only that single error in the cassandra.log.
>>
>>
>> On Thu, Aug 19, 2010 at 11:42 PM, Peter Schuller
>> <pe...@infidyne.com> wrote:
>>>
>>> > What is my "live set"?
>>>
>>> Sorry; that meant the "set of data acually live (i.e., not garbage) in
>>> the heap". In other words, the amount of memory truly "used".
>>>
>>> > Is the system CPU bound given the few statements
>>> > below? This is from running 4 concurrent processes against the
>>> > node...do I
>>> > need to throttle back the concurrent read/writers?
>>> >
>>> > I do all reads/writes as Quorum. (Replication factor of 3).
>>>
>>> With quorom and 0.6.4 I don't think unthrottled writes are expected to
>>> cause a problem.
>>>
>>> > The memtable threshold is the default of 256.
>>> >
>>> > All caching is turned off.
>>> >
>>> > The database is pretty small, maybe a few million keys (2-3) in 4 CFs.
>>> > The
>>> > key size is pretty small. Some of the rows are pretty fat though
>>> > (fatter
>>> > than I thought). I am saving secondary indexes in separate CFs and
>>> > those are
>>> > the large rows that I think might be part of the problem. I will
>>> > restart
>>> > testing turning these off and see if I see any difference.
>>> >
>>> > Would an extra fat row explain repeated OOM crashes in a row? I have
>>> > finally
>>> > got the system to stabilize relatively and I even ran compaction on the
>>> > bad
>>> > node without a problem (still no row size stats).
>>>
>>> Based on what you've said so far, the large rows are the only thing I
>>> would suspect may be the cause. With the amount of data and keys you
>>> say you have, you should definitely not be having memory issues with
>>> an 8 gig heap as a direct result of the data size/key count. A few
>>> million keys is not a lot at all; I still claim you should be able to
>>> handle hundreds of millions at least, from the perspective of bloom
>>> filters and such.
>>>
>>> So your plan to try it without these large rows is probably a good
>>> idea unless some else has a better idea.
>>>
>>> You may want to consider trying 0.7 betas too since it has removed the
>>> limitation with respect to large rows, assuming you do in fact want
>>> these large rows (see the CassandraLimitations wiki page that was
>>> posted earlier in this thread).
>>>
>>> > I now have several other nodes flapping with the following single error
>>> > in
>>> > the cassandra.log
>>> > Error: Exception thrown by the agent : java.lang.NullPointerException
>>> >
>>> > I assume this is an unrelated problem?
>>>
>>> Do you have a full stack trace?
>>>
>>> --
>>> / Peter Schuller
>>
>
>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
Is the need for 10k/sec/node just for bulk loading of data or is it
how your app will operate normally?  Those are very different things.

On Sun, Aug 22, 2010 at 4:11 AM, Wayne <wa...@gmail.com> wrote:
> Currently each node has 4x1TB SATA disks. In MySQL we have 15tb currently
> with no replication. To move this to Cassandra replication factor 3 we need
> 45TB assuming the space usage is the same, but it is probably more. We had
> assumed a 30 node cluster with 4tb per node would suffice with head room for
> compaction and to growth (120 TB).
>
> SSD drives for 30 nodes in this size range are not cost feasible for us. We
> can try to use 15k SAS drives and have more spindles but then our per node
> cost goes up. I guess I naively thought cassandra would do its magic and a
> few commodity SATA hard drives would be fine.
>
> Our performance requirement does not need 10k writes/node/sec 24 hours a
> day, but if we can not get really good performance the switch from MySQL
> becomes harder to rationalize. We can currently restore from a MySQL dump a
> 2.5 terabyte backup (plain old insert statements) in 4-5 days. I expect as
> much or more from cassandra and I feel years away from simply loading 2+tb
> into cassandra without so many issues.
>
> What is really required in hardware for a 100+tb cluster with near 10k/sec
> write performance sustained? If the answer is SSD what can be expected from
> 15k SAS drives and what from SATA?
>
> Thank you for your advice, I am struggling with how to make this work. Any
> insight you can provide would be greatly appreciated.
>
>
>
> On Sun, Aug 22, 2010 at 8:58 AM, Benjamin Black <b...@b3k.us> wrote:
>>
>> How much storage do you need?  240G SSDs quite capable of saturating a
>> 3Gbps SATA link are $600.  Larger ones are also available with similar
>> performance.  Perhaps you could share a bit more about the storage and
>> performance requirements.  How SSDs to sustain 10k writes/sec PER NODE
>> WITH LINEAR SCALING "breaks down the commodity server concept" eludes
>> me.
>>
>>
>> b
>>
>> On Sat, Aug 21, 2010 at 11:27 PM, Wayne <wa...@gmail.com> wrote:
>> > Thank you for the advice, I will try these settings. I am running
>> > defaults
>> > right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
>> > disks in raid 0 for the data.
>> >
>> > From your email you are implying this hardware can not handle this level
>> > of
>> > sustained writes? That kind of breaks down the commodity server concept
>> > for
>> > me. I have never used anything but a 15k SAS disk (fastest disk money
>> > could
>> > buy until SSD) ALWAYS with a database. I have tried to throw out that
>> > mentality here but are you saying nothing has really changed/ Spindles
>> > spindles spindles as fast as you can afford is what I have always
>> > known...I
>> > guess that applies here? Do I need to spend $10k per node instead of
>> > $3.5k
>> > to get SUSTAINED 10k writes/sec per node?
>> >
>> >
>> >
>> > On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
>> >>
>> >> My guess is that you have (at least) 2 problems right now:
>> >>
>> >> You are writing 10k ops/sec to each node, but have default memtable
>> >> flush settings.  This is resulting in memtable flushing every 30
>> >> seconds (default ops flush setting is 300k).  You thus have a
>> >> proliferation of tiny sstables and are seeing minor compactions
>> >> triggered every couple of minutes.
>> >>
>> >> You have started a major compaction which is now competing with those
>> >> near constant minor compactions for far too little I/O (3 SATA drives
>> >> in RAID0, perhaps?).  Normally, this would result in a massive
>> >> ballooning of your heap use as all sorts of activities (like memtable
>> >> flushes) backed up, as well.
>> >>
>> >> I suggest you increase the memtable flush ops to at least 10 (million)
>> >> if you are going to sustain that many writes/sec, along with an
>> >> increase in the flush MB to match, based on your typical bytes/write
>> >> op.  Long term, this level of write activity demands a lot faster
>> >> storage (iops and bandwidth).
>> >>
>> >>
>> >> b
>> >> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
>> >> > I am already running with those options. I thought maybe that is why
>> >> > they
>> >> > never get completed as they keep pushed pushed down in priority? I am
>> >> > getting timeouts now and then but for the most part the cluster keeps
>> >> > running. Is it normal/ok for the repair and compaction to take so
>> >> > long?
>> >> > It
>> >> > has been over 12 hours since they were submitted.
>> >> >
>> >> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> yes, the AES is the repair.
>> >> >>
>> >> >> if you are running linux, try adding the options to reduce
>> >> >> compaction
>> >> >> priority from
>> >> >> http://wiki.apache.org/cassandra/PerformanceTuning
>> >> >>
>> >> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> >> >> > I could tell from munin that the disk utilization was getting
>> >> >> > crazy
>> >> >> > high,
>> >> >> > but the strange thing is that it seemed to "stall". The
>> >> >> > utilization
>> >> >> > went
>> >> >> > way
>> >> >> > down and everything seemed to flatten out. Requests piled up and
>> >> >> > the
>> >> >> > node
>> >> >> > was doing nothing. It did not "crash" but was left in a useless
>> >> >> > state. I
>> >> >> > do
>> >> >> > not have access to the tpstats when that occurred. Attached is the
>> >> >> > munin
>> >> >> > chart, and you can see the flat line after Friday at noon.
>> >> >> >
>> >> >> > I have reduced the writers from 10 per to 8 per node and they seem
>> >> >> > to
>> >> >> > be
>> >> >> > still running, but I am afraid they are barely hanging on. I ran
>> >> >> > nodetool
>> >> >> > repair after rebooting the failed node and I do not think the
>> >> >> > repair
>> >> >> > ever
>> >> >> > completed. I also later ran compact on each node and some it
>> >> >> > finished
>> >> >> > but
>> >> >> > some it did not. Below is the tpstats currently for the node I had
>> >> >> > to
>> >> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued
>> >> >> > up?
>> >> >> > It
>> >> >> > seems several nodes are not getting enough free cycles to keep up.
>> >> >> > They
>> >> >> > are
>> >> >> > not timing out (30 sec timeout) for the most part but they are
>> >> >> > also
>> >> >> > not
>> >> >> > able
>> >> >> > to compact. Is this normal? Do I just give it time? I am migrating
>> >> >> > 2-3
>> >> >> > TB of
>> >> >> > data from Mysql so the load is constant and will be for days and
>> >> >> > it
>> >> >> > seems
>> >> >> > even with only 8 writer processes per node I am maxed out.
>> >> >> >
>> >> >> > Thanks for the advice. Any more pointers would be greatly
>> >> >> > appreciated.
>> >> >> >
>> >> >> > Pool Name                    Active   Pending      Completed
>> >> >> > FILEUTILS-DELETE-POOL             0         0           1868
>> >> >> > STREAM-STAGE                      1         1              2
>> >> >> > RESPONSE-STAGE                    0         2      769158645
>> >> >> > ROW-READ-STAGE                    0         0         140942
>> >> >> > LB-OPERATIONS                     0         0              0
>> >> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> >> >> > GMFD                              0         0         169712
>> >> >> > LB-TARGET                         0         0              0
>> >> >> > CONSISTENCY-MANAGER               0         0              0
>> >> >> > ROW-MUTATION-STAGE                0         1      865124937
>> >> >> > MESSAGE-STREAMING-POOL            0         0              6
>> >> >> > LOAD-BALANCER-STAGE               0         0              0
>> >> >> > FLUSH-SORTER-POOL                 0         0              0
>> >> >> > MEMTABLE-POST-FLUSHER             0         0           8088
>> >> >> > FLUSH-WRITER-POOL                 0         0           8088
>> >> >> > AE-SERVICE-STAGE                  1        34             54
>> >> >> > HINTED-HANDOFF-POOL               0         0              7
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
>> >> >> > wrote:
>> >> >> >>
>> >> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >> >> >>
>> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> >> > (1,078,378ms past timeout)
>> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> >> > (1,078,378ms past timeout)
>> >> >> >>
>> >> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
>> >> >> >> bogged
>> >> >> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> >> >> underlying cause is running out of disk bandwidth, well worth a
>> >> >> >> watch
>> >> >> >> http://riptano.blip.tv/file/4012133/).
>> >> >> >>
>> >> >> >> Can you send all of nodetool tpstats?
>> >> >> >>
>> >> >> >> Bill
>> >> >> >>
>> >> >> >
>> >> >> >
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Jonathan Ellis
>> >> >> Project Chair, Apache Cassandra
>> >> >> co-founder of Riptano, the source for professional Cassandra support
>> >> >> http://riptano.com
>> >> >
>> >> >
>> >
>> >
>
>

Re: Node OOM Problems

Posted by Edward Capriolo <ed...@gmail.com>.
On Sun, Aug 22, 2010 at 7:11 AM, Wayne <wa...@gmail.com> wrote:
> Currently each node has 4x1TB SATA disks. In MySQL we have 15tb currently
> with no replication. To move this to Cassandra replication factor 3 we need
> 45TB assuming the space usage is the same, but it is probably more. We had
> assumed a 30 node cluster with 4tb per node would suffice with head room for
> compaction and to growth (120 TB).
>
> SSD drives for 30 nodes in this size range are not cost feasible for us. We
> can try to use 15k SAS drives and have more spindles but then our per node
> cost goes up. I guess I naively thought cassandra would do its magic and a
> few commodity SATA hard drives would be fine.
>
> Our performance requirement does not need 10k writes/node/sec 24 hours a
> day, but if we can not get really good performance the switch from MySQL
> becomes harder to rationalize. We can currently restore from a MySQL dump a
> 2.5 terabyte backup (plain old insert statements) in 4-5 days. I expect as
> much or more from cassandra and I feel years away from simply loading 2+tb
> into cassandra without so many issues.
>
> What is really required in hardware for a 100+tb cluster with near 10k/sec
> write performance sustained? If the answer is SSD what can be expected from
> 15k SAS drives and what from SATA?
>
> Thank you for your advice, I am struggling with how to make this work. Any
> insight you can provide would be greatly appreciated.
>
>
>
> On Sun, Aug 22, 2010 at 8:58 AM, Benjamin Black <b...@b3k.us> wrote:
>>
>> How much storage do you need?  240G SSDs quite capable of saturating a
>> 3Gbps SATA link are $600.  Larger ones are also available with similar
>> performance.  Perhaps you could share a bit more about the storage and
>> performance requirements.  How SSDs to sustain 10k writes/sec PER NODE
>> WITH LINEAR SCALING "breaks down the commodity server concept" eludes
>> me.
>>
>>
>> b
>>
>> On Sat, Aug 21, 2010 at 11:27 PM, Wayne <wa...@gmail.com> wrote:
>> > Thank you for the advice, I will try these settings. I am running
>> > defaults
>> > right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
>> > disks in raid 0 for the data.
>> >
>> > From your email you are implying this hardware can not handle this level
>> > of
>> > sustained writes? That kind of breaks down the commodity server concept
>> > for
>> > me. I have never used anything but a 15k SAS disk (fastest disk money
>> > could
>> > buy until SSD) ALWAYS with a database. I have tried to throw out that
>> > mentality here but are you saying nothing has really changed/ Spindles
>> > spindles spindles as fast as you can afford is what I have always
>> > known...I
>> > guess that applies here? Do I need to spend $10k per node instead of
>> > $3.5k
>> > to get SUSTAINED 10k writes/sec per node?
>> >
>> >
>> >
>> > On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
>> >>
>> >> My guess is that you have (at least) 2 problems right now:
>> >>
>> >> You are writing 10k ops/sec to each node, but have default memtable
>> >> flush settings.  This is resulting in memtable flushing every 30
>> >> seconds (default ops flush setting is 300k).  You thus have a
>> >> proliferation of tiny sstables and are seeing minor compactions
>> >> triggered every couple of minutes.
>> >>
>> >> You have started a major compaction which is now competing with those
>> >> near constant minor compactions for far too little I/O (3 SATA drives
>> >> in RAID0, perhaps?).  Normally, this would result in a massive
>> >> ballooning of your heap use as all sorts of activities (like memtable
>> >> flushes) backed up, as well.
>> >>
>> >> I suggest you increase the memtable flush ops to at least 10 (million)
>> >> if you are going to sustain that many writes/sec, along with an
>> >> increase in the flush MB to match, based on your typical bytes/write
>> >> op.  Long term, this level of write activity demands a lot faster
>> >> storage (iops and bandwidth).
>> >>
>> >>
>> >> b
>> >> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
>> >> > I am already running with those options. I thought maybe that is why
>> >> > they
>> >> > never get completed as they keep pushed pushed down in priority? I am
>> >> > getting timeouts now and then but for the most part the cluster keeps
>> >> > running. Is it normal/ok for the repair and compaction to take so
>> >> > long?
>> >> > It
>> >> > has been over 12 hours since they were submitted.
>> >> >
>> >> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> yes, the AES is the repair.
>> >> >>
>> >> >> if you are running linux, try adding the options to reduce
>> >> >> compaction
>> >> >> priority from
>> >> >> http://wiki.apache.org/cassandra/PerformanceTuning
>> >> >>
>> >> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> >> >> > I could tell from munin that the disk utilization was getting
>> >> >> > crazy
>> >> >> > high,
>> >> >> > but the strange thing is that it seemed to "stall". The
>> >> >> > utilization
>> >> >> > went
>> >> >> > way
>> >> >> > down and everything seemed to flatten out. Requests piled up and
>> >> >> > the
>> >> >> > node
>> >> >> > was doing nothing. It did not "crash" but was left in a useless
>> >> >> > state. I
>> >> >> > do
>> >> >> > not have access to the tpstats when that occurred. Attached is the
>> >> >> > munin
>> >> >> > chart, and you can see the flat line after Friday at noon.
>> >> >> >
>> >> >> > I have reduced the writers from 10 per to 8 per node and they seem
>> >> >> > to
>> >> >> > be
>> >> >> > still running, but I am afraid they are barely hanging on. I ran
>> >> >> > nodetool
>> >> >> > repair after rebooting the failed node and I do not think the
>> >> >> > repair
>> >> >> > ever
>> >> >> > completed. I also later ran compact on each node and some it
>> >> >> > finished
>> >> >> > but
>> >> >> > some it did not. Below is the tpstats currently for the node I had
>> >> >> > to
>> >> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued
>> >> >> > up?
>> >> >> > It
>> >> >> > seems several nodes are not getting enough free cycles to keep up.
>> >> >> > They
>> >> >> > are
>> >> >> > not timing out (30 sec timeout) for the most part but they are
>> >> >> > also
>> >> >> > not
>> >> >> > able
>> >> >> > to compact. Is this normal? Do I just give it time? I am migrating
>> >> >> > 2-3
>> >> >> > TB of
>> >> >> > data from Mysql so the load is constant and will be for days and
>> >> >> > it
>> >> >> > seems
>> >> >> > even with only 8 writer processes per node I am maxed out.
>> >> >> >
>> >> >> > Thanks for the advice. Any more pointers would be greatly
>> >> >> > appreciated.
>> >> >> >
>> >> >> > Pool Name                    Active   Pending      Completed
>> >> >> > FILEUTILS-DELETE-POOL             0         0           1868
>> >> >> > STREAM-STAGE                      1         1              2
>> >> >> > RESPONSE-STAGE                    0         2      769158645
>> >> >> > ROW-READ-STAGE                    0         0         140942
>> >> >> > LB-OPERATIONS                     0         0              0
>> >> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> >> >> > GMFD                              0         0         169712
>> >> >> > LB-TARGET                         0         0              0
>> >> >> > CONSISTENCY-MANAGER               0         0              0
>> >> >> > ROW-MUTATION-STAGE                0         1      865124937
>> >> >> > MESSAGE-STREAMING-POOL            0         0              6
>> >> >> > LOAD-BALANCER-STAGE               0         0              0
>> >> >> > FLUSH-SORTER-POOL                 0         0              0
>> >> >> > MEMTABLE-POST-FLUSHER             0         0           8088
>> >> >> > FLUSH-WRITER-POOL                 0         0           8088
>> >> >> > AE-SERVICE-STAGE                  1        34             54
>> >> >> > HINTED-HANDOFF-POOL               0         0              7
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
>> >> >> > wrote:
>> >> >> >>
>> >> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >> >> >>
>> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> >> > (1,078,378ms past timeout)
>> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> >> > (1,078,378ms past timeout)
>> >> >> >>
>> >> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
>> >> >> >> bogged
>> >> >> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> >> >> underlying cause is running out of disk bandwidth, well worth a
>> >> >> >> watch
>> >> >> >> http://riptano.blip.tv/file/4012133/).
>> >> >> >>
>> >> >> >> Can you send all of nodetool tpstats?
>> >> >> >>
>> >> >> >> Bill
>> >> >> >>
>> >> >> >
>> >> >> >
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Jonathan Ellis
>> >> >> Project Chair, Apache Cassandra
>> >> >> co-founder of Riptano, the source for professional Cassandra support
>> >> >> http://riptano.com
>> >> >
>> >> >
>> >
>> >
>
>

>> Our performance requirement does not need 10k writes/node/sec 24 hours a day
If you major and minor compaction (s) are stepping on each other,
before your bulk load you can disable compaction through JMX, perform
your bulk load, and then re-enable compaction.

This make sense even if you are not having problems as you can avoid
compacting the same data multiple times during loading.

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
Currently each node has 4x1TB SATA disks. In MySQL we have 15tb currently
with no replication. To move this to Cassandra replication factor 3 we need
45TB assuming the space usage is the same, but it is probably more. We had
assumed a 30 node cluster with 4tb per node would suffice with head room for
compaction and to growth (120 TB).

SSD drives for 30 nodes in this size range are not cost feasible for us. We
can try to use 15k SAS drives and have more spindles but then our per node
cost goes up. I guess I naively thought cassandra would do its magic and a
few commodity SATA hard drives would be fine.

Our performance requirement does not *need* 10k writes/node/sec 24 hours a
day, but if we can not get really good performance the switch from MySQL
becomes harder to rationalize. We can currently restore from a MySQL dump a
2.5 terabyte backup (plain old insert statements) in 4-5 days. I expect as
much or more from cassandra and I feel years away from simply loading 2+tb
into cassandra without so many issues.

What is really required in hardware for a 100+tb cluster with near 10k/sec
write performance sustained? If the answer is SSD what can be expected from
15k SAS drives and what from SATA?

Thank you for your advice, I am struggling with how to make this work. Any
insight you can provide would be greatly appreciated.



On Sun, Aug 22, 2010 at 8:58 AM, Benjamin Black <b...@b3k.us> wrote:

> How much storage do you need?  240G SSDs quite capable of saturating a
> 3Gbps SATA link are $600.  Larger ones are also available with similar
> performance.  Perhaps you could share a bit more about the storage and
> performance requirements.  How SSDs to sustain 10k writes/sec PER NODE
> WITH LINEAR SCALING "breaks down the commodity server concept" eludes
> me.
>
>
> b
>
> On Sat, Aug 21, 2010 at 11:27 PM, Wayne <wa...@gmail.com> wrote:
> > Thank you for the advice, I will try these settings. I am running
> defaults
> > right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
> > disks in raid 0 for the data.
> >
> > From your email you are implying this hardware can not handle this level
> of
> > sustained writes? That kind of breaks down the commodity server concept
> for
> > me. I have never used anything but a 15k SAS disk (fastest disk money
> could
> > buy until SSD) ALWAYS with a database. I have tried to throw out that
> > mentality here but are you saying nothing has really changed/ Spindles
> > spindles spindles as fast as you can afford is what I have always
> known...I
> > guess that applies here? Do I need to spend $10k per node instead of
> $3.5k
> > to get SUSTAINED 10k writes/sec per node?
> >
> >
> >
> > On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
> >>
> >> My guess is that you have (at least) 2 problems right now:
> >>
> >> You are writing 10k ops/sec to each node, but have default memtable
> >> flush settings.  This is resulting in memtable flushing every 30
> >> seconds (default ops flush setting is 300k).  You thus have a
> >> proliferation of tiny sstables and are seeing minor compactions
> >> triggered every couple of minutes.
> >>
> >> You have started a major compaction which is now competing with those
> >> near constant minor compactions for far too little I/O (3 SATA drives
> >> in RAID0, perhaps?).  Normally, this would result in a massive
> >> ballooning of your heap use as all sorts of activities (like memtable
> >> flushes) backed up, as well.
> >>
> >> I suggest you increase the memtable flush ops to at least 10 (million)
> >> if you are going to sustain that many writes/sec, along with an
> >> increase in the flush MB to match, based on your typical bytes/write
> >> op.  Long term, this level of write activity demands a lot faster
> >> storage (iops and bandwidth).
> >>
> >>
> >> b
> >> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
> >> > I am already running with those options. I thought maybe that is why
> >> > they
> >> > never get completed as they keep pushed pushed down in priority? I am
> >> > getting timeouts now and then but for the most part the cluster keeps
> >> > running. Is it normal/ok for the repair and compaction to take so
> long?
> >> > It
> >> > has been over 12 hours since they were submitted.
> >> >
> >> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
> >> > wrote:
> >> >>
> >> >> yes, the AES is the repair.
> >> >>
> >> >> if you are running linux, try adding the options to reduce compaction
> >> >> priority from
> >> >> http://wiki.apache.org/cassandra/PerformanceTuning
> >> >>
> >> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
> >> >> > I could tell from munin that the disk utilization was getting crazy
> >> >> > high,
> >> >> > but the strange thing is that it seemed to "stall". The utilization
> >> >> > went
> >> >> > way
> >> >> > down and everything seemed to flatten out. Requests piled up and
> the
> >> >> > node
> >> >> > was doing nothing. It did not "crash" but was left in a useless
> >> >> > state. I
> >> >> > do
> >> >> > not have access to the tpstats when that occurred. Attached is the
> >> >> > munin
> >> >> > chart, and you can see the flat line after Friday at noon.
> >> >> >
> >> >> > I have reduced the writers from 10 per to 8 per node and they seem
> to
> >> >> > be
> >> >> > still running, but I am afraid they are barely hanging on. I ran
> >> >> > nodetool
> >> >> > repair after rebooting the failed node and I do not think the
> repair
> >> >> > ever
> >> >> > completed. I also later ran compact on each node and some it
> finished
> >> >> > but
> >> >> > some it did not. Below is the tpstats currently for the node I had
> to
> >> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued
> up?
> >> >> > It
> >> >> > seems several nodes are not getting enough free cycles to keep up.
> >> >> > They
> >> >> > are
> >> >> > not timing out (30 sec timeout) for the most part but they are also
> >> >> > not
> >> >> > able
> >> >> > to compact. Is this normal? Do I just give it time? I am migrating
> >> >> > 2-3
> >> >> > TB of
> >> >> > data from Mysql so the load is constant and will be for days and it
> >> >> > seems
> >> >> > even with only 8 writer processes per node I am maxed out.
> >> >> >
> >> >> > Thanks for the advice. Any more pointers would be greatly
> >> >> > appreciated.
> >> >> >
> >> >> > Pool Name                    Active   Pending      Completed
> >> >> > FILEUTILS-DELETE-POOL             0         0           1868
> >> >> > STREAM-STAGE                      1         1              2
> >> >> > RESPONSE-STAGE                    0         2      769158645
> >> >> > ROW-READ-STAGE                    0         0         140942
> >> >> > LB-OPERATIONS                     0         0              0
> >> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
> >> >> > GMFD                              0         0         169712
> >> >> > LB-TARGET                         0         0              0
> >> >> > CONSISTENCY-MANAGER               0         0              0
> >> >> > ROW-MUTATION-STAGE                0         1      865124937
> >> >> > MESSAGE-STREAMING-POOL            0         0              6
> >> >> > LOAD-BALANCER-STAGE               0         0              0
> >> >> > FLUSH-SORTER-POOL                 0         0              0
> >> >> > MEMTABLE-POST-FLUSHER             0         0           8088
> >> >> > FLUSH-WRITER-POOL                 0         0           8088
> >> >> > AE-SERVICE-STAGE                  1        34             54
> >> >> > HINTED-HANDOFF-POOL               0         0              7
> >> >> >
> >> >> >
> >> >> >
> >> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
> >> >> > wrote:
> >> >> >>
> >> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
> >> >> >>
> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
> >> >> >> > (1,078,378ms past timeout)
> >> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> >> >> > MessageDeserializationTask.java (line 47) dropping message
> >> >> >> > (1,078,378ms past timeout)
> >> >> >>
> >> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
> >> >> >> bogged
> >> >> >> downstream, (eg here's Ben Black describing the symptom when the
> >> >> >> underlying cause is running out of disk bandwidth, well worth a
> >> >> >> watch
> >> >> >> http://riptano.blip.tv/file/4012133/).
> >> >> >>
> >> >> >> Can you send all of nodetool tpstats?
> >> >> >>
> >> >> >> Bill
> >> >> >>
> >> >> >
> >> >> >
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Jonathan Ellis
> >> >> Project Chair, Apache Cassandra
> >> >> co-founder of Riptano, the source for professional Cassandra support
> >> >> http://riptano.com
> >> >
> >> >
> >
> >
>

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
How much storage do you need?  240G SSDs quite capable of saturating a
3Gbps SATA link are $600.  Larger ones are also available with similar
performance.  Perhaps you could share a bit more about the storage and
performance requirements.  How SSDs to sustain 10k writes/sec PER NODE
WITH LINEAR SCALING "breaks down the commodity server concept" eludes
me.


b

On Sat, Aug 21, 2010 at 11:27 PM, Wayne <wa...@gmail.com> wrote:
> Thank you for the advice, I will try these settings. I am running defaults
> right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
> disks in raid 0 for the data.
>
> From your email you are implying this hardware can not handle this level of
> sustained writes? That kind of breaks down the commodity server concept for
> me. I have never used anything but a 15k SAS disk (fastest disk money could
> buy until SSD) ALWAYS with a database. I have tried to throw out that
> mentality here but are you saying nothing has really changed/ Spindles
> spindles spindles as fast as you can afford is what I have always known...I
> guess that applies here? Do I need to spend $10k per node instead of $3.5k
> to get SUSTAINED 10k writes/sec per node?
>
>
>
> On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
>>
>> My guess is that you have (at least) 2 problems right now:
>>
>> You are writing 10k ops/sec to each node, but have default memtable
>> flush settings.  This is resulting in memtable flushing every 30
>> seconds (default ops flush setting is 300k).  You thus have a
>> proliferation of tiny sstables and are seeing minor compactions
>> triggered every couple of minutes.
>>
>> You have started a major compaction which is now competing with those
>> near constant minor compactions for far too little I/O (3 SATA drives
>> in RAID0, perhaps?).  Normally, this would result in a massive
>> ballooning of your heap use as all sorts of activities (like memtable
>> flushes) backed up, as well.
>>
>> I suggest you increase the memtable flush ops to at least 10 (million)
>> if you are going to sustain that many writes/sec, along with an
>> increase in the flush MB to match, based on your typical bytes/write
>> op.  Long term, this level of write activity demands a lot faster
>> storage (iops and bandwidth).
>>
>>
>> b
>> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
>> > I am already running with those options. I thought maybe that is why
>> > they
>> > never get completed as they keep pushed pushed down in priority? I am
>> > getting timeouts now and then but for the most part the cluster keeps
>> > running. Is it normal/ok for the repair and compaction to take so long?
>> > It
>> > has been over 12 hours since they were submitted.
>> >
>> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
>> > wrote:
>> >>
>> >> yes, the AES is the repair.
>> >>
>> >> if you are running linux, try adding the options to reduce compaction
>> >> priority from
>> >> http://wiki.apache.org/cassandra/PerformanceTuning
>> >>
>> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> >> > I could tell from munin that the disk utilization was getting crazy
>> >> > high,
>> >> > but the strange thing is that it seemed to "stall". The utilization
>> >> > went
>> >> > way
>> >> > down and everything seemed to flatten out. Requests piled up and the
>> >> > node
>> >> > was doing nothing. It did not "crash" but was left in a useless
>> >> > state. I
>> >> > do
>> >> > not have access to the tpstats when that occurred. Attached is the
>> >> > munin
>> >> > chart, and you can see the flat line after Friday at noon.
>> >> >
>> >> > I have reduced the writers from 10 per to 8 per node and they seem to
>> >> > be
>> >> > still running, but I am afraid they are barely hanging on. I ran
>> >> > nodetool
>> >> > repair after rebooting the failed node and I do not think the repair
>> >> > ever
>> >> > completed. I also later ran compact on each node and some it finished
>> >> > but
>> >> > some it did not. Below is the tpstats currently for the node I had to
>> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?
>> >> > It
>> >> > seems several nodes are not getting enough free cycles to keep up.
>> >> > They
>> >> > are
>> >> > not timing out (30 sec timeout) for the most part but they are also
>> >> > not
>> >> > able
>> >> > to compact. Is this normal? Do I just give it time? I am migrating
>> >> > 2-3
>> >> > TB of
>> >> > data from Mysql so the load is constant and will be for days and it
>> >> > seems
>> >> > even with only 8 writer processes per node I am maxed out.
>> >> >
>> >> > Thanks for the advice. Any more pointers would be greatly
>> >> > appreciated.
>> >> >
>> >> > Pool Name                    Active   Pending      Completed
>> >> > FILEUTILS-DELETE-POOL             0         0           1868
>> >> > STREAM-STAGE                      1         1              2
>> >> > RESPONSE-STAGE                    0         2      769158645
>> >> > ROW-READ-STAGE                    0         0         140942
>> >> > LB-OPERATIONS                     0         0              0
>> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> >> > GMFD                              0         0         169712
>> >> > LB-TARGET                         0         0              0
>> >> > CONSISTENCY-MANAGER               0         0              0
>> >> > ROW-MUTATION-STAGE                0         1      865124937
>> >> > MESSAGE-STREAMING-POOL            0         0              6
>> >> > LOAD-BALANCER-STAGE               0         0              0
>> >> > FLUSH-SORTER-POOL                 0         0              0
>> >> > MEMTABLE-POST-FLUSHER             0         0           8088
>> >> > FLUSH-WRITER-POOL                 0         0           8088
>> >> > AE-SERVICE-STAGE                  1        34             54
>> >> > HINTED-HANDOFF-POOL               0         0              7
>> >> >
>> >> >
>> >> >
>> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
>> >> > wrote:
>> >> >>
>> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >> >>
>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> > (1,078,378ms past timeout)
>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> > (1,078,378ms past timeout)
>> >> >>
>> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
>> >> >> bogged
>> >> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> >> underlying cause is running out of disk bandwidth, well worth a
>> >> >> watch
>> >> >> http://riptano.blip.tv/file/4012133/).
>> >> >>
>> >> >> Can you send all of nodetool tpstats?
>> >> >>
>> >> >> Bill
>> >> >>
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Jonathan Ellis
>> >> Project Chair, Apache Cassandra
>> >> co-founder of Riptano, the source for professional Cassandra support
>> >> http://riptano.com
>> >
>> >
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
Thanks for the confirmation this is NOT the way to go. I will stick with 4
disks raid 0 with a single data directory.


On Mon, Aug 23, 2010 at 9:24 PM, Rob Coli <rc...@digg.com> wrote:

> On 8/22/10 12:00 AM, Wayne wrote:
>
>> Due to compaction being so expensive in terms of disk resources, does it
>> make more sense to have 2 data volumes instead of one? We have 4 data
>> disks in raid 0, would this make more sense to be 2 x 2 disks in raid 0?
>> That way the reader and writer I assume would always be a different set
>> of spindles?
>>
>
> Cassandra supports multiple data directories via specifying multiple
> DataFileDirectory items in the DataFileDirectories block. Unfortunately due
> to the way that compaction currently works, this often results in one or the
> other drive being much hotter than the other and lots of inter-disk
> shuffling of data.
>
> https://issues.apache.org/jira/browse/CASSANDRA-897
>
> Is a JIRA which refers to these issues. I am not aware of anyone who has
> successfully improved their overall performance via multiple data
> directories. I am, however, aware of people who have encountered negative
> consequences as a result of doing so. As such, I do not recommend it.
>
> =Rob
>
>

Re: Node OOM Problems

Posted by Rob Coli <rc...@digg.com>.
On 8/22/10 12:00 AM, Wayne wrote:
> Due to compaction being so expensive in terms of disk resources, does it
> make more sense to have 2 data volumes instead of one? We have 4 data
> disks in raid 0, would this make more sense to be 2 x 2 disks in raid 0?
> That way the reader and writer I assume would always be a different set
> of spindles?

Cassandra supports multiple data directories via specifying multiple 
DataFileDirectory items in the DataFileDirectories block. Unfortunately 
due to the way that compaction currently works, this often results in 
one or the other drive being much hotter than the other and lots of 
inter-disk shuffling of data.

https://issues.apache.org/jira/browse/CASSANDRA-897

Is a JIRA which refers to these issues. I am not aware of anyone who has 
successfully improved their overall performance via multiple data 
directories. I am, however, aware of people who have encountered 
negative consequences as a result of doing so. As such, I do not 
recommend it.

=Rob


Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
I see no reason to make that assumption.  Cassandra currently has no
mechanism to alternate in that manner.  At the update rate you
require, you just need more disk io (bandwidth and iops).
Alternatively, you could use a bunch more, smaller nodes with the same
SATA RAID setup so they each take many fewer writes/sec, and so can
keep with compaction.

On Sun, Aug 22, 2010 at 12:00 AM, Wayne <wa...@gmail.com> wrote:
> Due to compaction being so expensive in terms of disk resources, does it
> make more sense to have 2 data volumes instead of one? We have 4 data disks
> in raid 0, would this make more sense to be 2 x 2 disks in raid 0? That way
> the reader and writer I assume would always be a different set of spindles?
>
> On Sun, Aug 22, 2010 at 8:27 AM, Wayne <wa...@gmail.com> wrote:
>>
>> Thank you for the advice, I will try these settings. I am running defaults
>> right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
>> disks in raid 0 for the data.
>>
>> From your email you are implying this hardware can not handle this level
>> of sustained writes? That kind of breaks down the commodity server concept
>> for me. I have never used anything but a 15k SAS disk (fastest disk money
>> could buy until SSD) ALWAYS with a database. I have tried to throw out that
>> mentality here but are you saying nothing has really changed/ Spindles
>> spindles spindles as fast as you can afford is what I have always known...I
>> guess that applies here? Do I need to spend $10k per node instead of $3.5k
>> to get SUSTAINED 10k writes/sec per node?
>>
>>
>>
>> On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
>>>
>>> My guess is that you have (at least) 2 problems right now:
>>>
>>> You are writing 10k ops/sec to each node, but have default memtable
>>> flush settings.  This is resulting in memtable flushing every 30
>>> seconds (default ops flush setting is 300k).  You thus have a
>>> proliferation of tiny sstables and are seeing minor compactions
>>> triggered every couple of minutes.
>>>
>>> You have started a major compaction which is now competing with those
>>> near constant minor compactions for far too little I/O (3 SATA drives
>>> in RAID0, perhaps?).  Normally, this would result in a massive
>>> ballooning of your heap use as all sorts of activities (like memtable
>>> flushes) backed up, as well.
>>>
>>> I suggest you increase the memtable flush ops to at least 10 (million)
>>> if you are going to sustain that many writes/sec, along with an
>>> increase in the flush MB to match, based on your typical bytes/write
>>> op.  Long term, this level of write activity demands a lot faster
>>> storage (iops and bandwidth).
>>>
>>>
>>> b
>>> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
>>> > I am already running with those options. I thought maybe that is why
>>> > they
>>> > never get completed as they keep pushed pushed down in priority? I am
>>> > getting timeouts now and then but for the most part the cluster keeps
>>> > running. Is it normal/ok for the repair and compaction to take so long?
>>> > It
>>> > has been over 12 hours since they were submitted.
>>> >
>>> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
>>> > wrote:
>>> >>
>>> >> yes, the AES is the repair.
>>> >>
>>> >> if you are running linux, try adding the options to reduce compaction
>>> >> priority from
>>> >> http://wiki.apache.org/cassandra/PerformanceTuning
>>> >>
>>> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>>> >> > I could tell from munin that the disk utilization was getting crazy
>>> >> > high,
>>> >> > but the strange thing is that it seemed to "stall". The utilization
>>> >> > went
>>> >> > way
>>> >> > down and everything seemed to flatten out. Requests piled up and the
>>> >> > node
>>> >> > was doing nothing. It did not "crash" but was left in a useless
>>> >> > state. I
>>> >> > do
>>> >> > not have access to the tpstats when that occurred. Attached is the
>>> >> > munin
>>> >> > chart, and you can see the flat line after Friday at noon.
>>> >> >
>>> >> > I have reduced the writers from 10 per to 8 per node and they seem
>>> >> > to be
>>> >> > still running, but I am afraid they are barely hanging on. I ran
>>> >> > nodetool
>>> >> > repair after rebooting the failed node and I do not think the repair
>>> >> > ever
>>> >> > completed. I also later ran compact on each node and some it
>>> >> > finished
>>> >> > but
>>> >> > some it did not. Below is the tpstats currently for the node I had
>>> >> > to
>>> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued
>>> >> > up?
>>> >> > It
>>> >> > seems several nodes are not getting enough free cycles to keep up.
>>> >> > They
>>> >> > are
>>> >> > not timing out (30 sec timeout) for the most part but they are also
>>> >> > not
>>> >> > able
>>> >> > to compact. Is this normal? Do I just give it time? I am migrating
>>> >> > 2-3
>>> >> > TB of
>>> >> > data from Mysql so the load is constant and will be for days and it
>>> >> > seems
>>> >> > even with only 8 writer processes per node I am maxed out.
>>> >> >
>>> >> > Thanks for the advice. Any more pointers would be greatly
>>> >> > appreciated.
>>> >> >
>>> >> > Pool Name                    Active   Pending      Completed
>>> >> > FILEUTILS-DELETE-POOL             0         0           1868
>>> >> > STREAM-STAGE                      1         1              2
>>> >> > RESPONSE-STAGE                    0         2      769158645
>>> >> > ROW-READ-STAGE                    0         0         140942
>>> >> > LB-OPERATIONS                     0         0              0
>>> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>>> >> > GMFD                              0         0         169712
>>> >> > LB-TARGET                         0         0              0
>>> >> > CONSISTENCY-MANAGER               0         0              0
>>> >> > ROW-MUTATION-STAGE                0         1      865124937
>>> >> > MESSAGE-STREAMING-POOL            0         0              6
>>> >> > LOAD-BALANCER-STAGE               0         0              0
>>> >> > FLUSH-SORTER-POOL                 0         0              0
>>> >> > MEMTABLE-POST-FLUSHER             0         0           8088
>>> >> > FLUSH-WRITER-POOL                 0         0           8088
>>> >> > AE-SERVICE-STAGE                  1        34             54
>>> >> > HINTED-HANDOFF-POOL               0         0              7
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
>>> >> > wrote:
>>> >> >>
>>> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>>> >> >>
>>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>>> >> >> > (1,078,378ms past timeout)
>>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>>> >> >> > (1,078,378ms past timeout)
>>> >> >>
>>> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
>>> >> >> bogged
>>> >> >> downstream, (eg here's Ben Black describing the symptom when the
>>> >> >> underlying cause is running out of disk bandwidth, well worth a
>>> >> >> watch
>>> >> >> http://riptano.blip.tv/file/4012133/).
>>> >> >>
>>> >> >> Can you send all of nodetool tpstats?
>>> >> >>
>>> >> >> Bill
>>> >> >>
>>> >> >
>>> >> >
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Jonathan Ellis
>>> >> Project Chair, Apache Cassandra
>>> >> co-founder of Riptano, the source for professional Cassandra support
>>> >> http://riptano.com
>>> >
>>> >
>>
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
Due to compaction being so expensive in terms of disk resources, does it
make more sense to have 2 data volumes instead of one? We have 4 data disks
in raid 0, would this make more sense to be 2 x 2 disks in raid 0? That way
the reader and writer I assume would always be a different set of spindles?

On Sun, Aug 22, 2010 at 8:27 AM, Wayne <wa...@gmail.com> wrote:

> Thank you for the advice, I will try these settings. I am running defaults
> right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
> disks in raid 0 for the data.
>
> From your email you are implying this hardware can not handle this level of
> sustained writes? That kind of breaks down the commodity server concept for
> me. I have never used anything but a 15k SAS disk (fastest disk money could
> buy until SSD) ALWAYS with a database. I have tried to throw out that
> mentality here but are you saying nothing has really changed/ Spindles
> spindles spindles as fast as you can afford is what I have always known...I
> guess that applies here? Do I need to spend $10k per node instead of $3.5k
> to get SUSTAINED 10k writes/sec per node?
>
>
>
>
> On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:
>
>> My guess is that you have (at least) 2 problems right now:
>>
>> You are writing 10k ops/sec to each node, but have default memtable
>> flush settings.  This is resulting in memtable flushing every 30
>> seconds (default ops flush setting is 300k).  You thus have a
>> proliferation of tiny sstables and are seeing minor compactions
>> triggered every couple of minutes.
>>
>> You have started a major compaction which is now competing with those
>> near constant minor compactions for far too little I/O (3 SATA drives
>> in RAID0, perhaps?).  Normally, this would result in a massive
>> ballooning of your heap use as all sorts of activities (like memtable
>> flushes) backed up, as well.
>>
>> I suggest you increase the memtable flush ops to at least 10 (million)
>> if you are going to sustain that many writes/sec, along with an
>> increase in the flush MB to match, based on your typical bytes/write
>> op.  Long term, this level of write activity demands a lot faster
>> storage (iops and bandwidth).
>>
>>
>> b
>> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
>> > I am already running with those options. I thought maybe that is why
>> they
>> > never get completed as they keep pushed pushed down in priority? I am
>> > getting timeouts now and then but for the most part the cluster keeps
>> > running. Is it normal/ok for the repair and compaction to take so long?
>> It
>> > has been over 12 hours since they were submitted.
>> >
>> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
>> wrote:
>> >>
>> >> yes, the AES is the repair.
>> >>
>> >> if you are running linux, try adding the options to reduce compaction
>> >> priority from
>> >> http://wiki.apache.org/cassandra/PerformanceTuning
>> >>
>> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> >> > I could tell from munin that the disk utilization was getting crazy
>> >> > high,
>> >> > but the strange thing is that it seemed to "stall". The utilization
>> went
>> >> > way
>> >> > down and everything seemed to flatten out. Requests piled up and the
>> >> > node
>> >> > was doing nothing. It did not "crash" but was left in a useless
>> state. I
>> >> > do
>> >> > not have access to the tpstats when that occurred. Attached is the
>> munin
>> >> > chart, and you can see the flat line after Friday at noon.
>> >> >
>> >> > I have reduced the writers from 10 per to 8 per node and they seem to
>> be
>> >> > still running, but I am afraid they are barely hanging on. I ran
>> >> > nodetool
>> >> > repair after rebooting the failed node and I do not think the repair
>> >> > ever
>> >> > completed. I also later ran compact on each node and some it finished
>> >> > but
>> >> > some it did not. Below is the tpstats currently for the node I had to
>> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?
>> >> > It
>> >> > seems several nodes are not getting enough free cycles to keep up.
>> They
>> >> > are
>> >> > not timing out (30 sec timeout) for the most part but they are also
>> not
>> >> > able
>> >> > to compact. Is this normal? Do I just give it time? I am migrating
>> 2-3
>> >> > TB of
>> >> > data from Mysql so the load is constant and will be for days and it
>> >> > seems
>> >> > even with only 8 writer processes per node I am maxed out.
>> >> >
>> >> > Thanks for the advice. Any more pointers would be greatly
>> appreciated.
>> >> >
>> >> > Pool Name                    Active   Pending      Completed
>> >> > FILEUTILS-DELETE-POOL             0         0           1868
>> >> > STREAM-STAGE                      1         1              2
>> >> > RESPONSE-STAGE                    0         2      769158645
>> >> > ROW-READ-STAGE                    0         0         140942
>> >> > LB-OPERATIONS                     0         0              0
>> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> >> > GMFD                              0         0         169712
>> >> > LB-TARGET                         0         0              0
>> >> > CONSISTENCY-MANAGER               0         0              0
>> >> > ROW-MUTATION-STAGE                0         1      865124937
>> >> > MESSAGE-STREAMING-POOL            0         0              6
>> >> > LOAD-BALANCER-STAGE               0         0              0
>> >> > FLUSH-SORTER-POOL                 0         0              0
>> >> > MEMTABLE-POST-FLUSHER             0         0           8088
>> >> > FLUSH-WRITER-POOL                 0         0           8088
>> >> > AE-SERVICE-STAGE                  1        34             54
>> >> > HINTED-HANDOFF-POOL               0         0              7
>> >> >
>> >> >
>> >> >
>> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
>> wrote:
>> >> >>
>> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >> >>
>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> > (1,078,378ms past timeout)
>> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> >> > (1,078,378ms past timeout)
>> >> >>
>> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
>> bogged
>> >> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> >> underlying cause is running out of disk bandwidth, well worth a
>> watch
>> >> >> http://riptano.blip.tv/file/4012133/).
>> >> >>
>> >> >> Can you send all of nodetool tpstats?
>> >> >>
>> >> >> Bill
>> >> >>
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Jonathan Ellis
>> >> Project Chair, Apache Cassandra
>> >> co-founder of Riptano, the source for professional Cassandra support
>> >> http://riptano.com
>> >
>> >
>>
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
Thank you for the advice, I will try these settings. I am running defaults
right now. The disk subsystem is one SATA disk for commitlog and 4 SATA
disks in raid 0 for the data.

>From your email you are implying this hardware can not handle this level of
sustained writes? That kind of breaks down the commodity server concept for
me. I have never used anything but a 15k SAS disk (fastest disk money could
buy until SSD) ALWAYS with a database. I have tried to throw out that
mentality here but are you saying nothing has really changed/ Spindles
spindles spindles as fast as you can afford is what I have always known...I
guess that applies here? Do I need to spend $10k per node instead of $3.5k
to get SUSTAINED 10k writes/sec per node?



On Sat, Aug 21, 2010 at 11:03 PM, Benjamin Black <b...@b3k.us> wrote:

> My guess is that you have (at least) 2 problems right now:
>
> You are writing 10k ops/sec to each node, but have default memtable
> flush settings.  This is resulting in memtable flushing every 30
> seconds (default ops flush setting is 300k).  You thus have a
> proliferation of tiny sstables and are seeing minor compactions
> triggered every couple of minutes.
>
> You have started a major compaction which is now competing with those
> near constant minor compactions for far too little I/O (3 SATA drives
> in RAID0, perhaps?).  Normally, this would result in a massive
> ballooning of your heap use as all sorts of activities (like memtable
> flushes) backed up, as well.
>
> I suggest you increase the memtable flush ops to at least 10 (million)
> if you are going to sustain that many writes/sec, along with an
> increase in the flush MB to match, based on your typical bytes/write
> op.  Long term, this level of write activity demands a lot faster
> storage (iops and bandwidth).
>
>
> b
> On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
> > I am already running with those options. I thought maybe that is why they
> > never get completed as they keep pushed pushed down in priority? I am
> > getting timeouts now and then but for the most part the cluster keeps
> > running. Is it normal/ok for the repair and compaction to take so long?
> It
> > has been over 12 hours since they were submitted.
> >
> > On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com>
> wrote:
> >>
> >> yes, the AES is the repair.
> >>
> >> if you are running linux, try adding the options to reduce compaction
> >> priority from
> >> http://wiki.apache.org/cassandra/PerformanceTuning
> >>
> >> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
> >> > I could tell from munin that the disk utilization was getting crazy
> >> > high,
> >> > but the strange thing is that it seemed to "stall". The utilization
> went
> >> > way
> >> > down and everything seemed to flatten out. Requests piled up and the
> >> > node
> >> > was doing nothing. It did not "crash" but was left in a useless state.
> I
> >> > do
> >> > not have access to the tpstats when that occurred. Attached is the
> munin
> >> > chart, and you can see the flat line after Friday at noon.
> >> >
> >> > I have reduced the writers from 10 per to 8 per node and they seem to
> be
> >> > still running, but I am afraid they are barely hanging on. I ran
> >> > nodetool
> >> > repair after rebooting the failed node and I do not think the repair
> >> > ever
> >> > completed. I also later ran compact on each node and some it finished
> >> > but
> >> > some it did not. Below is the tpstats currently for the node I had to
> >> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?
> >> > It
> >> > seems several nodes are not getting enough free cycles to keep up.
> They
> >> > are
> >> > not timing out (30 sec timeout) for the most part but they are also
> not
> >> > able
> >> > to compact. Is this normal? Do I just give it time? I am migrating 2-3
> >> > TB of
> >> > data from Mysql so the load is constant and will be for days and it
> >> > seems
> >> > even with only 8 writer processes per node I am maxed out.
> >> >
> >> > Thanks for the advice. Any more pointers would be greatly appreciated.
> >> >
> >> > Pool Name                    Active   Pending      Completed
> >> > FILEUTILS-DELETE-POOL             0         0           1868
> >> > STREAM-STAGE                      1         1              2
> >> > RESPONSE-STAGE                    0         2      769158645
> >> > ROW-READ-STAGE                    0         0         140942
> >> > LB-OPERATIONS                     0         0              0
> >> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
> >> > GMFD                              0         0         169712
> >> > LB-TARGET                         0         0              0
> >> > CONSISTENCY-MANAGER               0         0              0
> >> > ROW-MUTATION-STAGE                0         1      865124937
> >> > MESSAGE-STREAMING-POOL            0         0              6
> >> > LOAD-BALANCER-STAGE               0         0              0
> >> > FLUSH-SORTER-POOL                 0         0              0
> >> > MEMTABLE-POST-FLUSHER             0         0           8088
> >> > FLUSH-WRITER-POOL                 0         0           8088
> >> > AE-SERVICE-STAGE                  1        34             54
> >> > HINTED-HANDOFF-POOL               0         0              7
> >> >
> >> >
> >> >
> >> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net>
> wrote:
> >> >>
> >> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
> >> >>
> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> >> > MessageDeserializationTask.java (line 47) dropping message
> >> >> > (1,078,378ms past timeout)
> >> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> >> > MessageDeserializationTask.java (line 47) dropping message
> >> >> > (1,078,378ms past timeout)
> >> >>
> >> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are
> bogged
> >> >> downstream, (eg here's Ben Black describing the symptom when the
> >> >> underlying cause is running out of disk bandwidth, well worth a watch
> >> >> http://riptano.blip.tv/file/4012133/).
> >> >>
> >> >> Can you send all of nodetool tpstats?
> >> >>
> >> >> Bill
> >> >>
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Jonathan Ellis
> >> Project Chair, Apache Cassandra
> >> co-founder of Riptano, the source for professional Cassandra support
> >> http://riptano.com
> >
> >
>

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
On Sun, Aug 22, 2010 at 2:03 PM, Wayne <wa...@gmail.com> wrote:
> From a testing whether cassandra can take the load long term I do not see it
> as different. Yes bulk loading can be made faster using very different

Then you need far more IO, whether it comes form faster drives or more
nodes.  If you can achieve 10k writes/sec/node and linear scaling
without sharding in MySQL on cheap, commodity hardware then I am
impressed.

> methods, but my purpose is to test cassandra with a large volume of writes
> (and not to bulk load as efficiently as possible). I have scaled back to 5
> writer threads per node and still see 8k writes/sec/node. With the larger
> memory table settings we shall see how it goes. I have no idea how to change
> a JMX setting and prefer to use std options to be frank. For us this is

If you want best performance, you must tune the system appropriately.
If you want to use the base settings (which are intended for the 1G
max heap which is way too small for anything interesting), expect
suboptimal performance for your application.

> after all an evaluation of whether Cassandra can replace Mysql.
>
> I thank everyone for their help.
>
> On Sun, Aug 22, 2010 at 10:37 PM, Benjamin Black <b...@b3k.us> wrote:
>>
>> Wayne,
>>
>> Bulk loading this much data is a very different prospect from needing
>> to sustain that rate of updates indefinitely.  As was suggested
>> earlier, you likely need to tune things differently, including
>> disabling minor compactions during the bulk load, to make this work
>> efficiently.
>>
>>
>> b
>>
>> On Sun, Aug 22, 2010 at 12:40 PM, Wayne <wa...@gmail.com> wrote:
>> > Has anyone loaded 2+ terabytes of real data in one stretch into a
>> > cluster
>> > without bulk loading and without any problems? How long did it take?
>> > What
>> > kind of nodes were used? How many writes/sec/node can be sustained for
>> > 24+
>> > hours?
>> >
>> >
>> >
>> > On Sun, Aug 22, 2010 at 8:22 PM, Peter Schuller
>> > <pe...@infidyne.com> wrote:
>> >>
>> >> I only sifted recent history of this thread (for time reasons), but:
>> >>
>> >> > You have started a major compaction which is now competing with those
>> >> > near constant minor compactions for far too little I/O (3 SATA drives
>> >> > in RAID0, perhaps?).  Normally, this would result in a massive
>> >> > ballooning of your heap use as all sorts of activities (like memtable
>> >> > flushes) backed up, as well.
>> >>
>> >> AFAIK memtable flushing is unrelated to compaction in the sense that
>> >> they occur concurrently and don't block each other (except to the
>> >> extent that they truly do compete for e.g. disk or CPU resources).
>> >>
>> >> While small memtables do indeed mean more compaction activity in
>> >> total, the expensiveness of any given compaction should not be
>> >> severely affecting.
>> >>
>> >> As far as I can tell, the two primary effects of small memtable sizes
>> >> are:
>> >>
>> >> * An increase in total amount of compaction work done in total for a
>> >> given database size.
>> >> * An increase in the number of sstables that may accumulate while
>> >> larger compactions are running.
>> >> ** That in turn is particularly relevant because it can generate a lot
>> >> of seek-bound activity; consider for example range queries that end up
>> >> spanning 10 000 files on disk.
>> >>
>> >> If memtable flushes are not able to complete fast enough to cope with
>> >> write activity, even if that is the case only during concurrenct
>> >> compaction (for whatever reason), that suggests to me that write
>> >> activity is too high. Increasing memtable sizes may help on average
>> >> due to decreased compaction work, but I don't see why it would
>> >> significantly affect the performance one compactions *do* in fact run.
>> >>
>> >> With respect to timeouts on writes: I make no claims as to whether it
>> >> is expected, because I have not yet investigated, but I definitely see
>> >> sporadic slowness when benchmarking high-throughput writes on a
>> >> cassandra trunk snapshot somewhere between 0.6 and 0.7. This occurs
>> >> even when writing to a machine where the commit log and data
>> >> directories are both on separate RAID volumes that are battery backed
>> >> and should have no trouble eating write bursts (and the data is such
>> >> that one is CPU bound  rather than diskbound on average; so it only
>> >> needs to eat bursts).
>> >>
>> >> I've had to add re-try to the benchmarking tool (or else up the
>> >> timeout) because the default was not enough.
>> >>
>> >> I have not investigated exactly why this happens but it's an
>> >> interesting effect that as far as I can tell should not be there.
>> >> Haver other people done high-throughput writes (to the point of CPU
>> >> saturation) over extended periods of time while consistently seeing
>> >> low latencies (consistencty meaning never exceeding hundreds of ms
>> >> over several days)?
>> >>
>> >>
>> >> --
>> >> / Peter Schuller
>> >
>> >
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
>From a testing whether cassandra can take the load long term I do not see it
as different. Yes bulk loading can be made faster using very different
methods, but my purpose is to test cassandra with a large volume of writes
(and not to bulk load as efficiently as possible). I have scaled back to 5
writer threads per node and still see 8k writes/sec/node. With the larger
memory table settings we shall see how it goes. I have no idea how to change
a JMX setting and prefer to use std options to be frank. For us this is
after all an evaluation of whether Cassandra can replace Mysql.

I thank everyone for their help.

On Sun, Aug 22, 2010 at 10:37 PM, Benjamin Black <b...@b3k.us> wrote:

> Wayne,
>
> Bulk loading this much data is a very different prospect from needing
> to sustain that rate of updates indefinitely.  As was suggested
> earlier, you likely need to tune things differently, including
> disabling minor compactions during the bulk load, to make this work
> efficiently.
>
>
> b
>
> On Sun, Aug 22, 2010 at 12:40 PM, Wayne <wa...@gmail.com> wrote:
> > Has anyone loaded 2+ terabytes of real data in one stretch into a cluster
> > without bulk loading and without any problems? How long did it take? What
> > kind of nodes were used? How many writes/sec/node can be sustained for
> 24+
> > hours?
> >
> >
> >
> > On Sun, Aug 22, 2010 at 8:22 PM, Peter Schuller
> > <pe...@infidyne.com> wrote:
> >>
> >> I only sifted recent history of this thread (for time reasons), but:
> >>
> >> > You have started a major compaction which is now competing with those
> >> > near constant minor compactions for far too little I/O (3 SATA drives
> >> > in RAID0, perhaps?).  Normally, this would result in a massive
> >> > ballooning of your heap use as all sorts of activities (like memtable
> >> > flushes) backed up, as well.
> >>
> >> AFAIK memtable flushing is unrelated to compaction in the sense that
> >> they occur concurrently and don't block each other (except to the
> >> extent that they truly do compete for e.g. disk or CPU resources).
> >>
> >> While small memtables do indeed mean more compaction activity in
> >> total, the expensiveness of any given compaction should not be
> >> severely affecting.
> >>
> >> As far as I can tell, the two primary effects of small memtable sizes
> are:
> >>
> >> * An increase in total amount of compaction work done in total for a
> >> given database size.
> >> * An increase in the number of sstables that may accumulate while
> >> larger compactions are running.
> >> ** That in turn is particularly relevant because it can generate a lot
> >> of seek-bound activity; consider for example range queries that end up
> >> spanning 10 000 files on disk.
> >>
> >> If memtable flushes are not able to complete fast enough to cope with
> >> write activity, even if that is the case only during concurrenct
> >> compaction (for whatever reason), that suggests to me that write
> >> activity is too high. Increasing memtable sizes may help on average
> >> due to decreased compaction work, but I don't see why it would
> >> significantly affect the performance one compactions *do* in fact run.
> >>
> >> With respect to timeouts on writes: I make no claims as to whether it
> >> is expected, because I have not yet investigated, but I definitely see
> >> sporadic slowness when benchmarking high-throughput writes on a
> >> cassandra trunk snapshot somewhere between 0.6 and 0.7. This occurs
> >> even when writing to a machine where the commit log and data
> >> directories are both on separate RAID volumes that are battery backed
> >> and should have no trouble eating write bursts (and the data is such
> >> that one is CPU bound  rather than diskbound on average; so it only
> >> needs to eat bursts).
> >>
> >> I've had to add re-try to the benchmarking tool (or else up the
> >> timeout) because the default was not enough.
> >>
> >> I have not investigated exactly why this happens but it's an
> >> interesting effect that as far as I can tell should not be there.
> >> Haver other people done high-throughput writes (to the point of CPU
> >> saturation) over extended periods of time while consistently seeing
> >> low latencies (consistencty meaning never exceeding hundreds of ms
> >> over several days)?
> >>
> >>
> >> --
> >> / Peter Schuller
> >
> >
>

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
Wayne,

Bulk loading this much data is a very different prospect from needing
to sustain that rate of updates indefinitely.  As was suggested
earlier, you likely need to tune things differently, including
disabling minor compactions during the bulk load, to make this work
efficiently.


b

On Sun, Aug 22, 2010 at 12:40 PM, Wayne <wa...@gmail.com> wrote:
> Has anyone loaded 2+ terabytes of real data in one stretch into a cluster
> without bulk loading and without any problems? How long did it take? What
> kind of nodes were used? How many writes/sec/node can be sustained for 24+
> hours?
>
>
>
> On Sun, Aug 22, 2010 at 8:22 PM, Peter Schuller
> <pe...@infidyne.com> wrote:
>>
>> I only sifted recent history of this thread (for time reasons), but:
>>
>> > You have started a major compaction which is now competing with those
>> > near constant minor compactions for far too little I/O (3 SATA drives
>> > in RAID0, perhaps?).  Normally, this would result in a massive
>> > ballooning of your heap use as all sorts of activities (like memtable
>> > flushes) backed up, as well.
>>
>> AFAIK memtable flushing is unrelated to compaction in the sense that
>> they occur concurrently and don't block each other (except to the
>> extent that they truly do compete for e.g. disk or CPU resources).
>>
>> While small memtables do indeed mean more compaction activity in
>> total, the expensiveness of any given compaction should not be
>> severely affecting.
>>
>> As far as I can tell, the two primary effects of small memtable sizes are:
>>
>> * An increase in total amount of compaction work done in total for a
>> given database size.
>> * An increase in the number of sstables that may accumulate while
>> larger compactions are running.
>> ** That in turn is particularly relevant because it can generate a lot
>> of seek-bound activity; consider for example range queries that end up
>> spanning 10 000 files on disk.
>>
>> If memtable flushes are not able to complete fast enough to cope with
>> write activity, even if that is the case only during concurrenct
>> compaction (for whatever reason), that suggests to me that write
>> activity is too high. Increasing memtable sizes may help on average
>> due to decreased compaction work, but I don't see why it would
>> significantly affect the performance one compactions *do* in fact run.
>>
>> With respect to timeouts on writes: I make no claims as to whether it
>> is expected, because I have not yet investigated, but I definitely see
>> sporadic slowness when benchmarking high-throughput writes on a
>> cassandra trunk snapshot somewhere between 0.6 and 0.7. This occurs
>> even when writing to a machine where the commit log and data
>> directories are both on separate RAID volumes that are battery backed
>> and should have no trouble eating write bursts (and the data is such
>> that one is CPU bound  rather than diskbound on average; so it only
>> needs to eat bursts).
>>
>> I've had to add re-try to the benchmarking tool (or else up the
>> timeout) because the default was not enough.
>>
>> I have not investigated exactly why this happens but it's an
>> interesting effect that as far as I can tell should not be there.
>> Haver other people done high-throughput writes (to the point of CPU
>> saturation) over extended periods of time while consistently seeing
>> low latencies (consistencty meaning never exceeding hundreds of ms
>> over several days)?
>>
>>
>> --
>> / Peter Schuller
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
Has anyone loaded 2+ terabytes of real data in one stretch into a cluster
without bulk loading and without any problems? How long did it take? What
kind of nodes were used? How many writes/sec/node can be sustained for 24+
hours?



On Sun, Aug 22, 2010 at 8:22 PM, Peter Schuller <peter.schuller@infidyne.com
> wrote:

> I only sifted recent history of this thread (for time reasons), but:
>
> > You have started a major compaction which is now competing with those
> > near constant minor compactions for far too little I/O (3 SATA drives
> > in RAID0, perhaps?).  Normally, this would result in a massive
> > ballooning of your heap use as all sorts of activities (like memtable
> > flushes) backed up, as well.
>
> AFAIK memtable flushing is unrelated to compaction in the sense that
> they occur concurrently and don't block each other (except to the
> extent that they truly do compete for e.g. disk or CPU resources).
>
> While small memtables do indeed mean more compaction activity in
> total, the expensiveness of any given compaction should not be
> severely affecting.
>
> As far as I can tell, the two primary effects of small memtable sizes are:
>
> * An increase in total amount of compaction work done in total for a
> given database size.
> * An increase in the number of sstables that may accumulate while
> larger compactions are running.
> ** That in turn is particularly relevant because it can generate a lot
> of seek-bound activity; consider for example range queries that end up
> spanning 10 000 files on disk.
>
> If memtable flushes are not able to complete fast enough to cope with
> write activity, even if that is the case only during concurrenct
> compaction (for whatever reason), that suggests to me that write
> activity is too high. Increasing memtable sizes may help on average
> due to decreased compaction work, but I don't see why it would
> significantly affect the performance one compactions *do* in fact run.
>
> With respect to timeouts on writes: I make no claims as to whether it
> is expected, because I have not yet investigated, but I definitely see
> sporadic slowness when benchmarking high-throughput writes on a
> cassandra trunk snapshot somewhere between 0.6 and 0.7. This occurs
> even when writing to a machine where the commit log and data
> directories are both on separate RAID volumes that are battery backed
> and should have no trouble eating write bursts (and the data is such
> that one is CPU bound  rather than diskbound on average; so it only
> needs to eat bursts).
>
> I've had to add re-try to the benchmarking tool (or else up the
> timeout) because the default was not enough.
>
> I have not investigated exactly why this happens but it's an
> interesting effect that as far as I can tell should not be there.
> Haver other people done high-throughput writes (to the point of CPU
> saturation) over extended periods of time while consistently seeing
> low latencies (consistencty meaning never exceeding hundreds of ms
> over several days)?
>
>
> --
> / Peter Schuller
>

Re: Node OOM Problems

Posted by Peter Schuller <pe...@infidyne.com>.
I only sifted recent history of this thread (for time reasons), but:

> You have started a major compaction which is now competing with those
> near constant minor compactions for far too little I/O (3 SATA drives
> in RAID0, perhaps?).  Normally, this would result in a massive
> ballooning of your heap use as all sorts of activities (like memtable
> flushes) backed up, as well.

AFAIK memtable flushing is unrelated to compaction in the sense that
they occur concurrently and don't block each other (except to the
extent that they truly do compete for e.g. disk or CPU resources).

While small memtables do indeed mean more compaction activity in
total, the expensiveness of any given compaction should not be
severely affecting.

As far as I can tell, the two primary effects of small memtable sizes are:

* An increase in total amount of compaction work done in total for a
given database size.
* An increase in the number of sstables that may accumulate while
larger compactions are running.
** That in turn is particularly relevant because it can generate a lot
of seek-bound activity; consider for example range queries that end up
spanning 10 000 files on disk.

If memtable flushes are not able to complete fast enough to cope with
write activity, even if that is the case only during concurrenct
compaction (for whatever reason), that suggests to me that write
activity is too high. Increasing memtable sizes may help on average
due to decreased compaction work, but I don't see why it would
significantly affect the performance one compactions *do* in fact run.

With respect to timeouts on writes: I make no claims as to whether it
is expected, because I have not yet investigated, but I definitely see
sporadic slowness when benchmarking high-throughput writes on a
cassandra trunk snapshot somewhere between 0.6 and 0.7. This occurs
even when writing to a machine where the commit log and data
directories are both on separate RAID volumes that are battery backed
and should have no trouble eating write bursts (and the data is such
that one is CPU bound  rather than diskbound on average; so it only
needs to eat bursts).

I've had to add re-try to the benchmarking tool (or else up the
timeout) because the default was not enough.

I have not investigated exactly why this happens but it's an
interesting effect that as far as I can tell should not be there.
Haver other people done high-throughput writes (to the point of CPU
saturation) over extended periods of time while consistently seeing
low latencies (consistencty meaning never exceeding hundreds of ms
over several days)?


-- 
/ Peter Schuller

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
My guess is that you have (at least) 2 problems right now:

You are writing 10k ops/sec to each node, but have default memtable
flush settings.  This is resulting in memtable flushing every 30
seconds (default ops flush setting is 300k).  You thus have a
proliferation of tiny sstables and are seeing minor compactions
triggered every couple of minutes.

You have started a major compaction which is now competing with those
near constant minor compactions for far too little I/O (3 SATA drives
in RAID0, perhaps?).  Normally, this would result in a massive
ballooning of your heap use as all sorts of activities (like memtable
flushes) backed up, as well.

I suggest you increase the memtable flush ops to at least 10 (million)
if you are going to sustain that many writes/sec, along with an
increase in the flush MB to match, based on your typical bytes/write
op.  Long term, this level of write activity demands a lot faster
storage (iops and bandwidth).


b
On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
> I am already running with those options. I thought maybe that is why they
> never get completed as they keep pushed pushed down in priority? I am
> getting timeouts now and then but for the most part the cluster keeps
> running. Is it normal/ok for the repair and compaction to take so long? It
> has been over 12 hours since they were submitted.
>
> On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com> wrote:
>>
>> yes, the AES is the repair.
>>
>> if you are running linux, try adding the options to reduce compaction
>> priority from
>> http://wiki.apache.org/cassandra/PerformanceTuning
>>
>> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> > I could tell from munin that the disk utilization was getting crazy
>> > high,
>> > but the strange thing is that it seemed to "stall". The utilization went
>> > way
>> > down and everything seemed to flatten out. Requests piled up and the
>> > node
>> > was doing nothing. It did not "crash" but was left in a useless state. I
>> > do
>> > not have access to the tpstats when that occurred. Attached is the munin
>> > chart, and you can see the flat line after Friday at noon.
>> >
>> > I have reduced the writers from 10 per to 8 per node and they seem to be
>> > still running, but I am afraid they are barely hanging on. I ran
>> > nodetool
>> > repair after rebooting the failed node and I do not think the repair
>> > ever
>> > completed. I also later ran compact on each node and some it finished
>> > but
>> > some it did not. Below is the tpstats currently for the node I had to
>> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?
>> > It
>> > seems several nodes are not getting enough free cycles to keep up. They
>> > are
>> > not timing out (30 sec timeout) for the most part but they are also not
>> > able
>> > to compact. Is this normal? Do I just give it time? I am migrating 2-3
>> > TB of
>> > data from Mysql so the load is constant and will be for days and it
>> > seems
>> > even with only 8 writer processes per node I am maxed out.
>> >
>> > Thanks for the advice. Any more pointers would be greatly appreciated.
>> >
>> > Pool Name                    Active   Pending      Completed
>> > FILEUTILS-DELETE-POOL             0         0           1868
>> > STREAM-STAGE                      1         1              2
>> > RESPONSE-STAGE                    0         2      769158645
>> > ROW-READ-STAGE                    0         0         140942
>> > LB-OPERATIONS                     0         0              0
>> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> > GMFD                              0         0         169712
>> > LB-TARGET                         0         0              0
>> > CONSISTENCY-MANAGER               0         0              0
>> > ROW-MUTATION-STAGE                0         1      865124937
>> > MESSAGE-STREAMING-POOL            0         0              6
>> > LOAD-BALANCER-STAGE               0         0              0
>> > FLUSH-SORTER-POOL                 0         0              0
>> > MEMTABLE-POST-FLUSHER             0         0           8088
>> > FLUSH-WRITER-POOL                 0         0           8088
>> > AE-SERVICE-STAGE                  1        34             54
>> > HINTED-HANDOFF-POOL               0         0              7
>> >
>> >
>> >
>> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net> wrote:
>> >>
>> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >>
>> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> > (1,078,378ms past timeout)
>> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> > (1,078,378ms past timeout)
>> >>
>> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
>> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> underlying cause is running out of disk bandwidth, well worth a watch
>> >> http://riptano.blip.tv/file/4012133/).
>> >>
>> >> Can you send all of nodetool tpstats?
>> >>
>> >> Bill
>> >>
>> >
>> >
>>
>>
>>
>> --
>> Jonathan Ellis
>> Project Chair, Apache Cassandra
>> co-founder of Riptano, the source for professional Cassandra support
>> http://riptano.com
>
>

Re: Node OOM Problems

Posted by Benjamin Black <b...@b3k.us>.
Perhaps I missed it in one of the earlier emails, but what is your
disk subsystem config?

On Sat, Aug 21, 2010 at 2:18 AM, Wayne <wa...@gmail.com> wrote:
> I am already running with those options. I thought maybe that is why they
> never get completed as they keep pushed pushed down in priority? I am
> getting timeouts now and then but for the most part the cluster keeps
> running. Is it normal/ok for the repair and compaction to take so long? It
> has been over 12 hours since they were submitted.
>
> On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com> wrote:
>>
>> yes, the AES is the repair.
>>
>> if you are running linux, try adding the options to reduce compaction
>> priority from
>> http://wiki.apache.org/cassandra/PerformanceTuning
>>
>> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
>> > I could tell from munin that the disk utilization was getting crazy
>> > high,
>> > but the strange thing is that it seemed to "stall". The utilization went
>> > way
>> > down and everything seemed to flatten out. Requests piled up and the
>> > node
>> > was doing nothing. It did not "crash" but was left in a useless state. I
>> > do
>> > not have access to the tpstats when that occurred. Attached is the munin
>> > chart, and you can see the flat line after Friday at noon.
>> >
>> > I have reduced the writers from 10 per to 8 per node and they seem to be
>> > still running, but I am afraid they are barely hanging on. I ran
>> > nodetool
>> > repair after rebooting the failed node and I do not think the repair
>> > ever
>> > completed. I also later ran compact on each node and some it finished
>> > but
>> > some it did not. Below is the tpstats currently for the node I had to
>> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?
>> > It
>> > seems several nodes are not getting enough free cycles to keep up. They
>> > are
>> > not timing out (30 sec timeout) for the most part but they are also not
>> > able
>> > to compact. Is this normal? Do I just give it time? I am migrating 2-3
>> > TB of
>> > data from Mysql so the load is constant and will be for days and it
>> > seems
>> > even with only 8 writer processes per node I am maxed out.
>> >
>> > Thanks for the advice. Any more pointers would be greatly appreciated.
>> >
>> > Pool Name                    Active   Pending      Completed
>> > FILEUTILS-DELETE-POOL             0         0           1868
>> > STREAM-STAGE                      1         1              2
>> > RESPONSE-STAGE                    0         2      769158645
>> > ROW-READ-STAGE                    0         0         140942
>> > LB-OPERATIONS                     0         0              0
>> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
>> > GMFD                              0         0         169712
>> > LB-TARGET                         0         0              0
>> > CONSISTENCY-MANAGER               0         0              0
>> > ROW-MUTATION-STAGE                0         1      865124937
>> > MESSAGE-STREAMING-POOL            0         0              6
>> > LOAD-BALANCER-STAGE               0         0              0
>> > FLUSH-SORTER-POOL                 0         0              0
>> > MEMTABLE-POST-FLUSHER             0         0           8088
>> > FLUSH-WRITER-POOL                 0         0           8088
>> > AE-SERVICE-STAGE                  1        34             54
>> > HINTED-HANDOFF-POOL               0         0              7
>> >
>> >
>> >
>> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net> wrote:
>> >>
>> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>> >>
>> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> > (1,078,378ms past timeout)
>> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> >> > MessageDeserializationTask.java (line 47) dropping message
>> >> > (1,078,378ms past timeout)
>> >>
>> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
>> >> downstream, (eg here's Ben Black describing the symptom when the
>> >> underlying cause is running out of disk bandwidth, well worth a watch
>> >> http://riptano.blip.tv/file/4012133/).
>> >>
>> >> Can you send all of nodetool tpstats?
>> >>
>> >> Bill
>> >>
>> >
>> >
>>
>>
>>
>> --
>> Jonathan Ellis
>> Project Chair, Apache Cassandra
>> co-founder of Riptano, the source for professional Cassandra support
>> http://riptano.com
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
I am already running with those options. I thought maybe that is why they
never get completed as they keep pushed pushed down in priority? I am
getting timeouts now and then but for the most part the cluster keeps
running. Is it normal/ok for the repair and compaction to take so long? It
has been over 12 hours since they were submitted.

On Sat, Aug 21, 2010 at 10:56 AM, Jonathan Ellis <jb...@gmail.com> wrote:

> yes, the AES is the repair.
>
> if you are running linux, try adding the options to reduce compaction
> priority from
> http://wiki.apache.org/cassandra/PerformanceTuning
>
> On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
> > I could tell from munin that the disk utilization was getting crazy high,
> > but the strange thing is that it seemed to "stall". The utilization went
> way
> > down and everything seemed to flatten out. Requests piled up and the node
> > was doing nothing. It did not "crash" but was left in a useless state. I
> do
> > not have access to the tpstats when that occurred. Attached is the munin
> > chart, and you can see the flat line after Friday at noon.
> >
> > I have reduced the writers from 10 per to 8 per node and they seem to be
> > still running, but I am afraid they are barely hanging on. I ran nodetool
> > repair after rebooting the failed node and I do not think the repair ever
> > completed. I also later ran compact on each node and some it finished but
> > some it did not. Below is the tpstats currently for the node I had to
> > restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?  It
> > seems several nodes are not getting enough free cycles to keep up. They
> are
> > not timing out (30 sec timeout) for the most part but they are also not
> able
> > to compact. Is this normal? Do I just give it time? I am migrating 2-3 TB
> of
> > data from Mysql so the load is constant and will be for days and it seems
> > even with only 8 writer processes per node I am maxed out.
> >
> > Thanks for the advice. Any more pointers would be greatly appreciated.
> >
> > Pool Name                    Active   Pending      Completed
> > FILEUTILS-DELETE-POOL             0         0           1868
> > STREAM-STAGE                      1         1              2
> > RESPONSE-STAGE                    0         2      769158645
> > ROW-READ-STAGE                    0         0         140942
> > LB-OPERATIONS                     0         0              0
> > MESSAGE-DESERIALIZER-POOL         1         0     1470221842
> > GMFD                              0         0         169712
> > LB-TARGET                         0         0              0
> > CONSISTENCY-MANAGER               0         0              0
> > ROW-MUTATION-STAGE                0         1      865124937
> > MESSAGE-STREAMING-POOL            0         0              6
> > LOAD-BALANCER-STAGE               0         0              0
> > FLUSH-SORTER-POOL                 0         0              0
> > MEMTABLE-POST-FLUSHER             0         0           8088
> > FLUSH-WRITER-POOL                 0         0           8088
> > AE-SERVICE-STAGE                  1        34             54
> > HINTED-HANDOFF-POOL               0         0              7
> >
> >
> >
> > On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net> wrote:
> >>
> >> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
> >>
> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> > MessageDeserializationTask.java (line 47) dropping message
> >> > (1,078,378ms past timeout)
> >> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> >> > MessageDeserializationTask.java (line 47) dropping message
> >> > (1,078,378ms past timeout)
> >>
> >> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
> >> downstream, (eg here's Ben Black describing the symptom when the
> >> underlying cause is running out of disk bandwidth, well worth a watch
> >> http://riptano.blip.tv/file/4012133/).
> >>
> >> Can you send all of nodetool tpstats?
> >>
> >> Bill
> >>
> >
> >
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of Riptano, the source for professional Cassandra support
> http://riptano.com
>

Re: Node OOM Problems

Posted by Jonathan Ellis <jb...@gmail.com>.
yes, the AES is the repair.

if you are running linux, try adding the options to reduce compaction
priority from
http://wiki.apache.org/cassandra/PerformanceTuning

On Sat, Aug 21, 2010 at 3:17 AM, Wayne <wa...@gmail.com> wrote:
> I could tell from munin that the disk utilization was getting crazy high,
> but the strange thing is that it seemed to "stall". The utilization went way
> down and everything seemed to flatten out. Requests piled up and the node
> was doing nothing. It did not "crash" but was left in a useless state. I do
> not have access to the tpstats when that occurred. Attached is the munin
> chart, and you can see the flat line after Friday at noon.
>
> I have reduced the writers from 10 per to 8 per node and they seem to be
> still running, but I am afraid they are barely hanging on. I ran nodetool
> repair after rebooting the failed node and I do not think the repair ever
> completed. I also later ran compact on each node and some it finished but
> some it did not. Below is the tpstats currently for the node I had to
> restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?  It
> seems several nodes are not getting enough free cycles to keep up. They are
> not timing out (30 sec timeout) for the most part but they are also not able
> to compact. Is this normal? Do I just give it time? I am migrating 2-3 TB of
> data from Mysql so the load is constant and will be for days and it seems
> even with only 8 writer processes per node I am maxed out.
>
> Thanks for the advice. Any more pointers would be greatly appreciated.
>
> Pool Name                    Active   Pending      Completed
> FILEUTILS-DELETE-POOL             0         0           1868
> STREAM-STAGE                      1         1              2
> RESPONSE-STAGE                    0         2      769158645
> ROW-READ-STAGE                    0         0         140942
> LB-OPERATIONS                     0         0              0
> MESSAGE-DESERIALIZER-POOL         1         0     1470221842
> GMFD                              0         0         169712
> LB-TARGET                         0         0              0
> CONSISTENCY-MANAGER               0         0              0
> ROW-MUTATION-STAGE                0         1      865124937
> MESSAGE-STREAMING-POOL            0         0              6
> LOAD-BALANCER-STAGE               0         0              0
> FLUSH-SORTER-POOL                 0         0              0
> MEMTABLE-POST-FLUSHER             0         0           8088
> FLUSH-WRITER-POOL                 0         0           8088
> AE-SERVICE-STAGE                  1        34             54
> HINTED-HANDOFF-POOL               0         0              7
>
>
>
> On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net> wrote:
>>
>> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>>
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> > MessageDeserializationTask.java (line 47) dropping message
>> > (1,078,378ms past timeout)
>> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
>> > MessageDeserializationTask.java (line 47) dropping message
>> > (1,078,378ms past timeout)
>>
>> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
>> downstream, (eg here's Ben Black describing the symptom when the
>> underlying cause is running out of disk bandwidth, well worth a watch
>> http://riptano.blip.tv/file/4012133/).
>>
>> Can you send all of nodetool tpstats?
>>
>> Bill
>>
>
>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
I could tell from munin that the disk utilization was getting crazy high,
but the strange thing is that it seemed to "stall". The utilization went way
down and everything seemed to flatten out. Requests piled up and the node
was doing nothing. It did not "crash" but was left in a useless state. I do
not have access to the tpstats when that occurred. Attached is the munin
chart, and you can see the flat line after Friday at noon.

I have reduced the writers from 10 per to 8 per node and they seem to be
still running, but I am afraid they are barely hanging on. I ran nodetool
repair after rebooting the failed node and I do not think the repair ever
completed. I also later ran compact on each node and some it finished but
some it did not. Below is the tpstats currently for the node I had to
restart. Is the AE-SERVICE-STAGE the repair and compaction queued up?  It
seems several nodes are not getting enough free cycles to keep up. They are
not timing out (30 sec timeout) for the most part but they are also not able
to compact. Is this normal? Do I just give it time? I am migrating 2-3 TB of
data from Mysql so the load is constant and will be for days and it seems
even with only 8 writer processes per node I am maxed out.

Thanks for the advice. Any more pointers would be greatly appreciated.

Pool Name                    Active   Pending      Completed
FILEUTILS-DELETE-POOL             0         0           1868
STREAM-STAGE                      1         1              2
RESPONSE-STAGE                    0         2      769158645
ROW-READ-STAGE                    0         0         140942
LB-OPERATIONS                     0         0              0
MESSAGE-DESERIALIZER-POOL         1         0     1470221842
GMFD                              0         0         169712
LB-TARGET                         0         0              0
CONSISTENCY-MANAGER               0         0              0
ROW-MUTATION-STAGE                0         1      865124937
MESSAGE-STREAMING-POOL            0         0              6
LOAD-BALANCER-STAGE               0         0              0
FLUSH-SORTER-POOL                 0         0              0
MEMTABLE-POST-FLUSHER             0         0           8088
FLUSH-WRITER-POOL                 0         0           8088
AE-SERVICE-STAGE                  1        34             54
HINTED-HANDOFF-POOL               0         0              7



On Fri, Aug 20, 2010 at 11:56 PM, Bill de hÓra <bi...@dehora.net> wrote:

> On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:
>
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> > MessageDeserializationTask.java (line 47) dropping message
> > (1,078,378ms past timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> > MessageDeserializationTask.java (line 47) dropping message
> > (1,078,378ms past timeout)
>
> MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
> downstream, (eg here's Ben Black describing the symptom when the
> underlying cause is running out of disk bandwidth, well worth a watch
> http://riptano.blip.tv/file/4012133/).
>
> Can you send all of nodetool tpstats?
>
> Bill
>
>

Re: Node OOM Problems

Posted by Bill de hÓra <bi...@dehora.net>.
On Fri, 2010-08-20 at 19:17 +0200, Wayne wrote:

>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message
> (1,078,378ms past timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message
> (1,078,378ms past timeout)

MESSAGE-DESERIALIZER-POOL usually backs up when other stages are bogged
downstream, (eg here's Ben Black describing the symptom when the
underlying cause is running out of disk bandwidth, well worth a watch
http://riptano.blip.tv/file/4012133/). 

Can you send all of nodetool tpstats?

Bill


Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
I deleted ALL data and reset the nodes from scratch. There are no more large
rows in there. 8-9megs MAX across all nodes. This appears to be a new
problem. I restarted the node in question and it seems to be running fine,
but I had to run repair on it as it appears to be missing a lot of data.


On Fri, Aug 20, 2010 at 7:51 PM, Edward Capriolo <ed...@gmail.com>wrote:

> On Fri, Aug 20, 2010 at 1:17 PM, Wayne <wa...@gmail.com> wrote:
> > I turned off the creation of the secondary indexes which had the large
> rows
> > and all seemed good. Thank you for the help. I was getting
> > 60k+/writes/second on the 6 node cluster.
> >
> > Unfortunately again three hours later a node went down. I can not even
> look
> > at the logs when it started since they are gone/recycled due to millions
> of
> > message deserialization messages. What are these? The node had 12,098,067
> > pending message-deserializer-pool entries in tpstats. The node was up
> > according to some nodes and down according to others which made it
> flapping
> > and still trying to take requests. What is the log warning message
> > deserialization task dropped message? Why would a node have 12 million of
> > these?
> >
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> > MessageDeserializationTask.java (line 47) dropping message (1,078,378ms
> past
> > timeout)
> >  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> > MessageDeserializationTask.java (line 47) dropping message (1,078,378ms
> past
> > timeout)
> >
> > I do not think this is a large row problem any more. All nodes show a max
> > row size around 8-9 megs.
> >
> > I looked at the munin charts and the disk IO seems to have spiked along
> with
> > compaction. Could compaction kicking in cause this? I have added the 3
> JVM
> > settings to make compaction a lower priority. Did this help cause this to
> > happen by slowing down and building up compaction on a heavily loaded
> > system?
> >
> > Thanks in advance for any help someone can provide.
> >
> >
> > On Fri, Aug 20, 2010 at 8:34 AM, Wayne <wa...@gmail.com> wrote:
> >>
> >> The NullPointerException does not crash the node. It only makes it
> flap/go
> >> down a for short period and then it comes back up. I do not see anything
> >> abnormal in the system log, only that single error in the cassandra.log.
> >>
> >>
> >> On Thu, Aug 19, 2010 at 11:42 PM, Peter Schuller
> >> <pe...@infidyne.com> wrote:
> >>>
> >>> > What is my "live set"?
> >>>
> >>> Sorry; that meant the "set of data acually live (i.e., not garbage) in
> >>> the heap". In other words, the amount of memory truly "used".
> >>>
> >>> > Is the system CPU bound given the few statements
> >>> > below? This is from running 4 concurrent processes against the
> >>> > node...do I
> >>> > need to throttle back the concurrent read/writers?
> >>> >
> >>> > I do all reads/writes as Quorum. (Replication factor of 3).
> >>>
> >>> With quorom and 0.6.4 I don't think unthrottled writes are expected to
> >>> cause a problem.
> >>>
> >>> > The memtable threshold is the default of 256.
> >>> >
> >>> > All caching is turned off.
> >>> >
> >>> > The database is pretty small, maybe a few million keys (2-3) in 4
> CFs.
> >>> > The
> >>> > key size is pretty small. Some of the rows are pretty fat though
> >>> > (fatter
> >>> > than I thought). I am saving secondary indexes in separate CFs and
> >>> > those are
> >>> > the large rows that I think might be part of the problem. I will
> >>> > restart
> >>> > testing turning these off and see if I see any difference.
> >>> >
> >>> > Would an extra fat row explain repeated OOM crashes in a row? I have
> >>> > finally
> >>> > got the system to stabilize relatively and I even ran compaction on
> the
> >>> > bad
> >>> > node without a problem (still no row size stats).
> >>>
> >>> Based on what you've said so far, the large rows are the only thing I
> >>> would suspect may be the cause. With the amount of data and keys you
> >>> say you have, you should definitely not be having memory issues with
> >>> an 8 gig heap as a direct result of the data size/key count. A few
> >>> million keys is not a lot at all; I still claim you should be able to
> >>> handle hundreds of millions at least, from the perspective of bloom
> >>> filters and such.
> >>>
> >>> So your plan to try it without these large rows is probably a good
> >>> idea unless some else has a better idea.
> >>>
> >>> You may want to consider trying 0.7 betas too since it has removed the
> >>> limitation with respect to large rows, assuming you do in fact want
> >>> these large rows (see the CassandraLimitations wiki page that was
> >>> posted earlier in this thread).
> >>>
> >>> > I now have several other nodes flapping with the following single
> error
> >>> > in
> >>> > the cassandra.log
> >>> > Error: Exception thrown by the agent : java.lang.NullPointerException
> >>> >
> >>> > I assume this is an unrelated problem?
> >>>
> >>> Do you have a full stack trace?
> >>>
> >>> --
> >>> / Peter Schuller
> >>
> >
> >
>
> Just because you are no longer creating the big rows does not mean
> they are no longer effecting you. For example, periodic compaction may
> run on those keys. Did you delete the keys, and run a major compaction
> to clear the data and tombstones?
>

Re: Node OOM Problems

Posted by Edward Capriolo <ed...@gmail.com>.
On Fri, Aug 20, 2010 at 1:17 PM, Wayne <wa...@gmail.com> wrote:
> I turned off the creation of the secondary indexes which had the large rows
> and all seemed good. Thank you for the help. I was getting
> 60k+/writes/second on the 6 node cluster.
>
> Unfortunately again three hours later a node went down. I can not even look
> at the logs when it started since they are gone/recycled due to millions of
> message deserialization messages. What are these? The node had 12,098,067
> pending message-deserializer-pool entries in tpstats. The node was up
> according to some nodes and down according to others which made it flapping
> and still trying to take requests. What is the log warning message
> deserialization task dropped message? Why would a node have 12 million of
> these?
>
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
> timeout)
>  WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
> MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
> timeout)
>
> I do not think this is a large row problem any more. All nodes show a max
> row size around 8-9 megs.
>
> I looked at the munin charts and the disk IO seems to have spiked along with
> compaction. Could compaction kicking in cause this? I have added the 3 JVM
> settings to make compaction a lower priority. Did this help cause this to
> happen by slowing down and building up compaction on a heavily loaded
> system?
>
> Thanks in advance for any help someone can provide.
>
>
> On Fri, Aug 20, 2010 at 8:34 AM, Wayne <wa...@gmail.com> wrote:
>>
>> The NullPointerException does not crash the node. It only makes it flap/go
>> down a for short period and then it comes back up. I do not see anything
>> abnormal in the system log, only that single error in the cassandra.log.
>>
>>
>> On Thu, Aug 19, 2010 at 11:42 PM, Peter Schuller
>> <pe...@infidyne.com> wrote:
>>>
>>> > What is my "live set"?
>>>
>>> Sorry; that meant the "set of data acually live (i.e., not garbage) in
>>> the heap". In other words, the amount of memory truly "used".
>>>
>>> > Is the system CPU bound given the few statements
>>> > below? This is from running 4 concurrent processes against the
>>> > node...do I
>>> > need to throttle back the concurrent read/writers?
>>> >
>>> > I do all reads/writes as Quorum. (Replication factor of 3).
>>>
>>> With quorom and 0.6.4 I don't think unthrottled writes are expected to
>>> cause a problem.
>>>
>>> > The memtable threshold is the default of 256.
>>> >
>>> > All caching is turned off.
>>> >
>>> > The database is pretty small, maybe a few million keys (2-3) in 4 CFs.
>>> > The
>>> > key size is pretty small. Some of the rows are pretty fat though
>>> > (fatter
>>> > than I thought). I am saving secondary indexes in separate CFs and
>>> > those are
>>> > the large rows that I think might be part of the problem. I will
>>> > restart
>>> > testing turning these off and see if I see any difference.
>>> >
>>> > Would an extra fat row explain repeated OOM crashes in a row? I have
>>> > finally
>>> > got the system to stabilize relatively and I even ran compaction on the
>>> > bad
>>> > node without a problem (still no row size stats).
>>>
>>> Based on what you've said so far, the large rows are the only thing I
>>> would suspect may be the cause. With the amount of data and keys you
>>> say you have, you should definitely not be having memory issues with
>>> an 8 gig heap as a direct result of the data size/key count. A few
>>> million keys is not a lot at all; I still claim you should be able to
>>> handle hundreds of millions at least, from the perspective of bloom
>>> filters and such.
>>>
>>> So your plan to try it without these large rows is probably a good
>>> idea unless some else has a better idea.
>>>
>>> You may want to consider trying 0.7 betas too since it has removed the
>>> limitation with respect to large rows, assuming you do in fact want
>>> these large rows (see the CassandraLimitations wiki page that was
>>> posted earlier in this thread).
>>>
>>> > I now have several other nodes flapping with the following single error
>>> > in
>>> > the cassandra.log
>>> > Error: Exception thrown by the agent : java.lang.NullPointerException
>>> >
>>> > I assume this is an unrelated problem?
>>>
>>> Do you have a full stack trace?
>>>
>>> --
>>> / Peter Schuller
>>
>
>

Just because you are no longer creating the big rows does not mean
they are no longer effecting you. For example, periodic compaction may
run on those keys. Did you delete the keys, and run a major compaction
to clear the data and tombstones?

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
I turned off the creation of the secondary indexes which had the large rows
and all seemed good. Thank you for the help. I was getting
60k+/writes/second on the 6 node cluster.

Unfortunately again three hours later a node went down. I can not even look
at the logs when it started since they are gone/recycled due to millions of
message deserialization messages. What are these? The node had 12,098,067
pending message-deserializer-pool entries in tpstats. The node was up
according to some nodes and down according to others which made it flapping
and still trying to take requests. What is the log warning message
deserialization task dropped message? Why would a node have 12 million of
these?

 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
timeout)
 WARN [MESSAGE-DESERIALIZER-POOL:1] 2010-08-20 16:57:02,602
MessageDeserializationTask.java (line 47) dropping message (1,078,378ms past
timeout)

I do not think this is a large row problem any more. All nodes show a max
row size around 8-9 megs.

I looked at the munin charts and the disk IO seems to have spiked along with
compaction. Could compaction kicking in cause this? I have added the 3 JVM
settings to make compaction a lower priority. Did this help cause this to
happen by slowing down and building up compaction on a heavily loaded
system?

Thanks in advance for any help someone can provide.


On Fri, Aug 20, 2010 at 8:34 AM, Wayne <wa...@gmail.com> wrote:

> The NullPointerException does not crash the node. It only makes it flap/go
> down a for short period and then it comes back up. I do not see anything
> abnormal in the system log, only that single error in the cassandra.log.
>
>
>
> On Thu, Aug 19, 2010 at 11:42 PM, Peter Schuller <
> peter.schuller@infidyne.com> wrote:
>
>> > What is my "live set"?
>>
>> Sorry; that meant the "set of data acually live (i.e., not garbage) in
>> the heap". In other words, the amount of memory truly "used".
>>
>> > Is the system CPU bound given the few statements
>> > below? This is from running 4 concurrent processes against the node...do
>> I
>> > need to throttle back the concurrent read/writers?
>> >
>> > I do all reads/writes as Quorum. (Replication factor of 3).
>>
>> With quorom and 0.6.4 I don't think unthrottled writes are expected to
>> cause a problem.
>>
>> > The memtable threshold is the default of 256.
>> >
>> > All caching is turned off.
>> >
>> > The database is pretty small, maybe a few million keys (2-3) in 4 CFs.
>> The
>> > key size is pretty small. Some of the rows are pretty fat though (fatter
>> > than I thought). I am saving secondary indexes in separate CFs and those
>> are
>> > the large rows that I think might be part of the problem. I will restart
>> > testing turning these off and see if I see any difference.
>> >
>> > Would an extra fat row explain repeated OOM crashes in a row? I have
>> finally
>> > got the system to stabilize relatively and I even ran compaction on the
>> bad
>> > node without a problem (still no row size stats).
>>
>> Based on what you've said so far, the large rows are the only thing I
>> would suspect may be the cause. With the amount of data and keys you
>> say you have, you should definitely not be having memory issues with
>> an 8 gig heap as a direct result of the data size/key count. A few
>> million keys is not a lot at all; I still claim you should be able to
>> handle hundreds of millions at least, from the perspective of bloom
>> filters and such.
>>
>> So your plan to try it without these large rows is probably a good
>> idea unless some else has a better idea.
>>
>> You may want to consider trying 0.7 betas too since it has removed the
>> limitation with respect to large rows, assuming you do in fact want
>> these large rows (see the CassandraLimitations wiki page that was
>> posted earlier in this thread).
>>
>> > I now have several other nodes flapping with the following single error
>> in
>> > the cassandra.log
>> > Error: Exception thrown by the agent : java.lang.NullPointerException
>> >
>> > I assume this is an unrelated problem?
>>
>> Do you have a full stack trace?
>>
>> --
>> / Peter Schuller
>>
>
>

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
The NullPointerException does not crash the node. It only makes it flap/go
down a for short period and then it comes back up. I do not see anything
abnormal in the system log, only that single error in the cassandra.log.


On Thu, Aug 19, 2010 at 11:42 PM, Peter Schuller <
peter.schuller@infidyne.com> wrote:

> > What is my "live set"?
>
> Sorry; that meant the "set of data acually live (i.e., not garbage) in
> the heap". In other words, the amount of memory truly "used".
>
> > Is the system CPU bound given the few statements
> > below? This is from running 4 concurrent processes against the node...do
> I
> > need to throttle back the concurrent read/writers?
> >
> > I do all reads/writes as Quorum. (Replication factor of 3).
>
> With quorom and 0.6.4 I don't think unthrottled writes are expected to
> cause a problem.
>
> > The memtable threshold is the default of 256.
> >
> > All caching is turned off.
> >
> > The database is pretty small, maybe a few million keys (2-3) in 4 CFs.
> The
> > key size is pretty small. Some of the rows are pretty fat though (fatter
> > than I thought). I am saving secondary indexes in separate CFs and those
> are
> > the large rows that I think might be part of the problem. I will restart
> > testing turning these off and see if I see any difference.
> >
> > Would an extra fat row explain repeated OOM crashes in a row? I have
> finally
> > got the system to stabilize relatively and I even ran compaction on the
> bad
> > node without a problem (still no row size stats).
>
> Based on what you've said so far, the large rows are the only thing I
> would suspect may be the cause. With the amount of data and keys you
> say you have, you should definitely not be having memory issues with
> an 8 gig heap as a direct result of the data size/key count. A few
> million keys is not a lot at all; I still claim you should be able to
> handle hundreds of millions at least, from the perspective of bloom
> filters and such.
>
> So your plan to try it without these large rows is probably a good
> idea unless some else has a better idea.
>
> You may want to consider trying 0.7 betas too since it has removed the
> limitation with respect to large rows, assuming you do in fact want
> these large rows (see the CassandraLimitations wiki page that was
> posted earlier in this thread).
>
> > I now have several other nodes flapping with the following single error
> in
> > the cassandra.log
> > Error: Exception thrown by the agent : java.lang.NullPointerException
> >
> > I assume this is an unrelated problem?
>
> Do you have a full stack trace?
>
> --
> / Peter Schuller
>

Re: Node OOM Problems

Posted by Peter Schuller <pe...@infidyne.com>.
> Sorry; that meant the "set of data acually live (i.e., not garbage) in
> the heap". In other words, the amount of memory truly "used".

And to clarify further this is not the same as the 'used' reported by
GC statistics, except as printed after a CMS concurrent mark/sweep has
completed (and even then the number is not exact due to changes made
during the concurrent mark/sweep phase).

That's why I said "live" (a term often used in the context of GC; an
object which is "live" is one which is reachable by the application
and thus by definition not eligible for garbage collection).

-- 
/ Peter Schuller

Re: Node OOM Problems

Posted by Peter Schuller <pe...@infidyne.com>.
> What is my "live set"?

Sorry; that meant the "set of data acually live (i.e., not garbage) in
the heap". In other words, the amount of memory truly "used".

> Is the system CPU bound given the few statements
> below? This is from running 4 concurrent processes against the node...do I
> need to throttle back the concurrent read/writers?
>
> I do all reads/writes as Quorum. (Replication factor of 3).

With quorom and 0.6.4 I don't think unthrottled writes are expected to
cause a problem.

> The memtable threshold is the default of 256.
>
> All caching is turned off.
>
> The database is pretty small, maybe a few million keys (2-3) in 4 CFs. The
> key size is pretty small. Some of the rows are pretty fat though (fatter
> than I thought). I am saving secondary indexes in separate CFs and those are
> the large rows that I think might be part of the problem. I will restart
> testing turning these off and see if I see any difference.
>
> Would an extra fat row explain repeated OOM crashes in a row? I have finally
> got the system to stabilize relatively and I even ran compaction on the bad
> node without a problem (still no row size stats).

Based on what you've said so far, the large rows are the only thing I
would suspect may be the cause. With the amount of data and keys you
say you have, you should definitely not be having memory issues with
an 8 gig heap as a direct result of the data size/key count. A few
million keys is not a lot at all; I still claim you should be able to
handle hundreds of millions at least, from the perspective of bloom
filters and such.

So your plan to try it without these large rows is probably a good
idea unless some else has a better idea.

You may want to consider trying 0.7 betas too since it has removed the
limitation with respect to large rows, assuming you do in fact want
these large rows (see the CassandraLimitations wiki page that was
posted earlier in this thread).

> I now have several other nodes flapping with the following single error in
> the cassandra.log
> Error: Exception thrown by the agent : java.lang.NullPointerException
>
> I assume this is an unrelated problem?

Do you have a full stack trace?

-- 
/ Peter Schuller

Re: Node OOM Problems

Posted by Wayne <wa...@gmail.com>.
What is my "live set"? Is the system CPU bound given the few statements
below? This is from running 4 concurrent processes against the node...do I
need to throttle back the concurrent read/writers?

I do all reads/writes as Quorum. (Replication factor of 3).

The memtable threshold is the default of 256.

All caching is turned off.

The database is pretty small, maybe a few million keys (2-3) in 4 CFs. The
key size is pretty small. Some of the rows are pretty fat though (fatter
than I thought). I am saving secondary indexes in separate CFs and those are
the large rows that I think might be part of the problem. I will restart
testing turning these off and see if I see any difference.

Would an extra fat row explain repeated OOM crashes in a row? I have finally
got the system to stabilize relatively and I even ran compaction on the bad
node without a problem (still no row size stats).

I now have several other nodes flapping with the following single error in
the cassandra.log
Error: Exception thrown by the agent : java.lang.NullPointerException

I assume this is an unrelated problem?

Thanks for all of your help!

On Thu, Aug 19, 2010 at 10:26 PM, Peter Schuller <
peter.schuller@infidyne.com> wrote:

> So, these:
>
> >  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line 116)
> GC
> > for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving 8326856720
> > used; max is 8700035072
> [snip]
> > INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116)
> GC for ConcurrentMarkSweep: 37122 ms, 157488
> > reclaimed leaving 8342836376 used; max is 8700035072
>
> ...show that you're live set is indeed very close to heap maximum, and
> so concurrent mark/sweep phases run often freeing very little memory.
> In addition the fact that it seems to take 35-45 seconds to do the
> concurrent mark/sweep on an 8 gig heap on a modern system suggests
> that you are probably CPU bound in cassandra at the time (meaning GC
> is slower).
>
> In short you're using too much memory in comparison to the maximum
> heap size. The expected result is to either get an OOM, or just become
> too slow due to excessive GC activity (usually the latter followed by
> the former).
>
> Now, the question is what memory is used *for*, and why. First off, to
> get that out of the way, are you inserting with consistency level
> ZERO? I am not sure whether it applies to 0.6.4 or not but there used
> to be an issue involving writes at consistency level ZERO not being
> throttled at all, meaning that if you threw writes at the system
> faster than it would handle them, you would accumulate memory use. I
> don't believe this is a problem with CL.ONE and above, even in 0.6.4
> (but someone correct me if I'm wrong).
>
> (As an aside: I'm not sure whether the behavior was such that it might
> explain OOM on restart as a result of accumulated commitlogs that get
> replayed faster than memtable flushing happens. Perhaps not, not
> sure.)
>
> In any case, the most important factors are what you're actually doing
> with the cluster, but you don't say much about the data. In particular
> how many rows and colums you're populating it with.
>
> The primary users of large amounts of memory in cassandra include
> (hopefully I'm not missing something major);
>
> * bloom filters that are used to efficiently avoid doing I/O on
> sstables that do not contain relevant data. the size of bloom filters
> scale linearly with the number of row keys (not columns right? I don't
> remember). so here we have an expected permanent, but low, memory use
> as a result of a large database. how large is your database? 100
> million keys? 1 billion? 10 billion?
>
> * the memtables; the currently active memtable and any memtables
> currently undergoing flushing. the size of these are directly
> controllable in the configuration file. make sure they are reasonable.
> (If you're not sure at all, with an 8 gig heap I'd say <= 512 mb is a
> reasonable recommendation unless you have a reason to make them
> larger)
>
> * row cache and key cache, both controllable in the configuration. in
> particular the row cache can be huge if you have configured it as
> such.
>
> * to some extent unflushed commitlogs; the commit log rotation
> threshold controls this. the default value is low enough that it
> should not be your culprit
>
> So the question is what you're usage is like. How many unique rows do
> you have? How many columns? The data size in and of itself should not
> matter much to memory use, except of course that extremely large
> individual values will be relevant to transient high memory use when
> they are read/written.
>
> In general, lacking large row caches and such things, you should be
> able to have hundreds of millions of entries on an 8 gb heap, assuming
> reasonably sized keys.
>
> --
> / Peter Schuller
>

Re: Node OOM Problems

Posted by Peter Schuller <pe...@infidyne.com>.
So, these:

>  INFO [GC inspection] 2010-08-19 16:34:46,656 GCInspector.java (line 116) GC
> for ConcurrentMarkSweep: 41615 ms, 192522712 reclaimed leaving 8326856720
> used; max is 8700035072
[snip]
> INFO [GC inspection] 2010-08-19 16:36:00,786 GCInspector.java (line 116) GC for ConcurrentMarkSweep: 37122 ms, 157488
> reclaimed leaving 8342836376 used; max is 8700035072

...show that you're live set is indeed very close to heap maximum, and
so concurrent mark/sweep phases run often freeing very little memory.
In addition the fact that it seems to take 35-45 seconds to do the
concurrent mark/sweep on an 8 gig heap on a modern system suggests
that you are probably CPU bound in cassandra at the time (meaning GC
is slower).

In short you're using too much memory in comparison to the maximum
heap size. The expected result is to either get an OOM, or just become
too slow due to excessive GC activity (usually the latter followed by
the former).

Now, the question is what memory is used *for*, and why. First off, to
get that out of the way, are you inserting with consistency level
ZERO? I am not sure whether it applies to 0.6.4 or not but there used
to be an issue involving writes at consistency level ZERO not being
throttled at all, meaning that if you threw writes at the system
faster than it would handle them, you would accumulate memory use. I
don't believe this is a problem with CL.ONE and above, even in 0.6.4
(but someone correct me if I'm wrong).

(As an aside: I'm not sure whether the behavior was such that it might
explain OOM on restart as a result of accumulated commitlogs that get
replayed faster than memtable flushing happens. Perhaps not, not
sure.)

In any case, the most important factors are what you're actually doing
with the cluster, but you don't say much about the data. In particular
how many rows and colums you're populating it with.

The primary users of large amounts of memory in cassandra include
(hopefully I'm not missing something major);

* bloom filters that are used to efficiently avoid doing I/O on
sstables that do not contain relevant data. the size of bloom filters
scale linearly with the number of row keys (not columns right? I don't
remember). so here we have an expected permanent, but low, memory use
as a result of a large database. how large is your database? 100
million keys? 1 billion? 10 billion?

* the memtables; the currently active memtable and any memtables
currently undergoing flushing. the size of these are directly
controllable in the configuration file. make sure they are reasonable.
(If you're not sure at all, with an 8 gig heap I'd say <= 512 mb is a
reasonable recommendation unless you have a reason to make them
larger)

* row cache and key cache, both controllable in the configuration. in
particular the row cache can be huge if you have configured it as
such.

* to some extent unflushed commitlogs; the commit log rotation
threshold controls this. the default value is low enough that it
should not be your culprit

So the question is what you're usage is like. How many unique rows do
you have? How many columns? The data size in and of itself should not
matter much to memory use, except of course that extremely large
individual values will be relevant to transient high memory use when
they are read/written.

In general, lacking large row caches and such things, you should be
able to have hundreds of millions of entries on an 8 gb heap, assuming
reasonably sized keys.

-- 
/ Peter Schuller