[
https://issues.apache.org/jira/browse/CASSANDRA-7373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14028391#comment-14028391
]
Jeff Griffith commented on CASSANDRA-7373:
------------------------------------------
Thanks Mikhail. If my explanation for the exception above makes sense, it would
seem the only mystery is how the "high-traffic" table got to be over half of
MAX-INT. In the logs below, is this amount about some crazy-large single
mutation or about all values in memory for this column family?
WARN [COMMIT-LOG-WRITER] 2014-06-11 20:25:29,505 CommitLog.java (line 349)
Skipping commitlog append of extremely large mutation (1833599403 bytes)
INFO [OptionalTasks:1] 2014-06-11 20:25:29,832 MeteredFlusher.java (line 64)
flushing high-traffic column family CFS(Keyspace='SyncCore',
ColumnFamily='EmailNetworkDeltas') (estimated 1836720026 bytes)
INFO [OptionalTasks:1] 2014-06-11 20:25:30,325 ColumnFamilyStore.java (line
633) Enqueuing flush of
Memtable-EmailNetworkDeltas@79278257(1837683252/1864225045 serialized/live
bytes, 1098 ops)
INFO [FlushWriter:1520] 2014-06-11 20:25:30,326 Memtable.java (line 398)
Writing Memtable-EmailNetworkDeltas@79278257(1837683252/1864225045
serialized/live bytes, 1098 ops)
ERROR [FlushWriter:1520] 2014-06-11 20:25:30,344 CassandraDaemon.java (line
191) Exception in thread Thread[FlushWriter:1520,5,main]
java.lang.NegativeArraySizeException
at
org.apache.cassandra.io.util.FastByteArrayOutputStream.expand(FastByteArrayOutputStream.java:104)
> Commit logs no longer deleting and MemtablePostFlusher pending growing
> ----------------------------------------------------------------------
>
> Key: CASSANDRA-7373
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7373
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: RHEL 6.5
> Cassandra 1.12.16
> Replication factor of 3
> Reporter: Francois Richard
>
> 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:
> {code}
> 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
> {code}
> Here is a potential error in the logs that can explain this:
> {code}
> 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)
> {code}
--
This message was sent by Atlassian JIRA
(v6.2#6252)