Details
Description
We have this issue where once in a while, we get into a situation where the MemtablePostFlusher is not executing and the space used by the commit logs on disks keeps on increasing and increasing.
We can observe the problem by invoking nodetool tpstats:
Pool Name Active Pending Completed Blocked All time blocked ReadStage 6 6 46650213 0 0 RequestResponseStage 0 0 130547421 0 0 MutationStage 2 2 116813206 0 0 ReadRepairStage 0 0 2322201 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 120780 0 0 AntiEntropyStage 0 0 0 0 0 MigrationStage 0 0 0 0 0 MemoryMeter 0 0 456 0 0 MemtablePostFlusher 1 447 6344 0 0 FlushWriter 0 0 6132 0 62 MiscStage 0 0 0 0 0 PendingRangeCalculator 0 0 6 0 0 commitlog_archiver 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 HintedHandoff 2 2 4 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 BINARY 0 READ 0 MUTATION 0 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0
Here is a potential error in the logs that can explain this:
ERROR [FlushWriter:2693] 2014-06-09 22:05:38,452 CassandraDaemon.java (line 191) Exception in thread Thread[FlushWriter:2693,5,main]
java.lang.NegativeArraySizeException
at org.apache.cassandra.io.util.FastByteArrayOutputStream.expand(FastByteArrayOutputStream.java:104)
at org.apache.cassandra.io.util.FastByteArrayOutputStream.write(FastByteArrayOutputStream.java:220)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
at org.apache.cassandra.io.util.DataOutputBuffer.write(DataOutputBuffer.java:60)
at org.apache.cassandra.utils.ByteBufferUtil.write(ByteBufferUtil.java:328)
at org.apache.cassandra.utils.ByteBufferUtil.writeWithLength(ByteBufferUtil.java:315)
at org.apache.cassandra.db.ColumnSerializer.serialize(ColumnSerializer.java:55)
at org.apache.cassandra.db.ColumnSerializer.serialize(ColumnSerializer.java:30)
at org.apache.cassandra.db.OnDiskAtom$Serializer.serializeForSSTable(OnDiskAtom.java:62)
at org.apache.cassandra.db.ColumnIndex$Builder.add(ColumnIndex.java:181)
at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:133)
at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:185)
at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:430)
at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:385)
at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
Attachments
Attachments
Issue Links
- relates to
-
CASSANDRA-7275 Errors in FlushRunnable may leave threads hung
- Resolved