[ 
https://issues.apache.org/jira/browse/CASSANDRA-7373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14028187#comment-14028187
 ] 

Jeff Griffith commented on CASSANDRA-7373:
------------------------------------------

Adding a bit more detail to this... A warning comes on the COMMIT-LOG-WRITER 
thread saying it is skipping the append for a large row:

 INFO [FlushWriter:3571] 2014-06-11 10:19:59,696 Memtable.java (line 436) 
Completed flushing 
/home/y/var/cassandra/data/SyncCore/CommEvents/SyncCore-CommEvents-ic-84518-Data.db
 (3647831 bytes) for commitlog position ReplayPosition(segmentId=1402438439770, 
position=61789)
 INFO [FlushWriter:3566] 2014-06-11 10:19:59,727 Memtable.java (line 436) 
Completed flushing 
/home/y/var/cassandra/data/SyncCore/RecentComms/SyncCore-RecentComms-ic-83750-Data.db
 (5619016 bytes) for commitlog position ReplayPosition(segmentId=1402438439770, 
position=61789)
 INFO [GossipStage:1] 2014-06-11 10:20:07,302 Gossiper.java (line 823) 
InetAddress /10.198.26.179 is now DOWN
 INFO [OptionalTasks:1] 2014-06-11 10:20:09,754 ColumnFamilyStore.java (line 
633) Enqueuing flush of 
Memtable-SingleEndpointNetworks@1949516166(19390313/37748736 serialized/live 
bytes, 37981 ops)
 INFO [FlushWriter:3568] 2014-06-11 10:20:09,755 Memtable.java (line 398) 
Writing Memtable-SingleEndpointNetworks@1949516166(19390313/37748736 
serialized/live bytes, 37981 ops)
 INFO [OptionalTasks:1] 2014-06-11 10:20:09,755 ColumnFamilyStore.java (line 
633) Enqueuing flush of 
Memtable-EmailNetworkDeltas@1578090988(14498572/14502911 serialized/live bytes, 
4587 ops)
 INFO [FlushWriter:3567] 2014-06-11 10:20:09,756 Memtable.java (line 398) 
Writing Memtable-EmailNetworkDeltas@1578090988(14498572/14502911 
serialized/live bytes, 4587 ops)
 INFO [FlushWriter:3567] 2014-06-11 10:20:09,789 Memtable.java (line 436) 
Completed flushing 
/home/y/var/cassandra/data/SyncCore/EmailNetworkDeltas/SyncCore-EmailNetworkDeltas-ic-90590-Data.db
 (95318 bytes) for commitlog position ReplayPosition(segmentId=1402438439771, 
position=43352)
 INFO [FlushWriter:3568] 2014-06-11 10:20:09,985 Memtable.java (line 436) 
Completed flushing 
/home/y/var/cassandra/data/SyncCore/SingleEndpointNetworks/SyncCore-SingleEndpointNetworks-ic-88034-Data.db
 (9499480 bytes) for commitlog position ReplayPosition(segmentId=1402438439771, 
position=43352)
 INFO [OptionalTasks:1] 2014-06-11 10:20:32,805 MeteredFlusher.java (line 64) 
flushing high-traffic column family CFS(Keyspace='SyncCore', 
ColumnFamily='EndpointPrefixIndexMinimized') (estimated 650749167 bytes)
 INFO [OptionalTasks:1] 2014-06-11 10:20:32,806 ColumnFamilyStore.java (line 
633) Enqueuing flush of 
Memtable-EndpointPrefixIndexMinimized@1634472324(74779192/650749167 
serialized/live bytes, 3059166 ops)
 INFO [FlushWriter:3573] 2014-06-11 10:20:32,807 Memtable.java (line 398) 
Writing Memtable-EndpointPrefixIndexMinimized@1634472324(74779192/650749167 
serialized/live bytes, 3059166 ops)
 INFO [FlushWriter:3573] 2014-06-11 10:20:36,459 Memtable.java (line 436) 
Completed flushing 
/home/y/var/cassandra/data/SyncCore/EndpointPrefixIndexMinimized/SyncCore-EndpointPrefixIndexMinimized-ic-127891-Data.db
 (34290509 bytes) for commitlog position 
ReplayPosition(segmentId=1402438439773, position=7326447)
 WARN [COMMIT-LOG-WRITER] 2014-06-11 10:20:52,605 CommitLog.java (line 349) 
Skipping commitlog append of extremely large mutation (1886300377 bytes)
 INFO [OptionalTasks:1] 2014-06-11 10:20:52,902 MeteredFlusher.java (line 64) 
flushing high-traffic column family CFS(Keyspace='SyncCore', 
ColumnFamily='EmailNetworkDeltas') (estimated 1886275564 bytes)
 INFO [OptionalTasks:1] 2014-06-11 10:20:53,452 ColumnFamilyStore.java (line 
633) Enqueuing flush of 
Memtable-EmailNetworkDeltas@745275182(1886662001/1887226753 serialized/live 
bytes, 474 ops)
 INFO [FlushWriter:3577] 2014-06-11 10:20:53,452 Memtable.java (line 398) 
Writing Memtable-EmailNetworkDeltas@745275182(1886662001/1887226753 
serialized/live bytes, 474 ops)
ERROR [FlushWriter:3577] 2014-06-11 10:20:53,463 CassandraDaemon.java (line 
191) Exception in thread Thread[FlushWriter:3577,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)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
 INFO [GossipStage:1] 2014-06-11 10:20:56,829 Gossiper.java (line 823) 
InetAddress /10.198.26.171 is now DOWN
 INFO [OptionalTasks:1] 2014-06-11 10:21:09,289 ColumnFamilyStore.java (line 
633) Enqueuing flush of Memtable-ListPrefixIndex@1419303241(498848/1765043 
serialized/live bytes, 1130 ops)
 INFO [FlushWriter:3579] 2014-06-11 10:21:09,290 Memtable.java (line 398) 
Writing Memtable-ListPrefixIndex@1419303241(498848/1765043 serialized/live 
bytes, 1130 ops)

> 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)

Reply via email to