[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-13 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/13/14 5:10 PM:
---

[~mishail] [~jbellis]  Is there already a separate bug for the second patch 
where the extend fails or should I create one?Or perhaps this one (7373) 
serves the purpose and 7275 fixes the CDL problem.


was (Author: jeffery.griffith):
[~mishail] [~jbellis]  Is there already a separate bug for the second patch 
where the extend fails or should I create one?

 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
Assignee: Mikhail Stepura
 Fix For: 1.2.17, 2.0.9, 2.1.0

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 0002-Handle-possible-integer-overflow.patch, 7373-v2.txt


 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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver0 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)
  

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-13 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/13/14 5:11 PM:
---

[~mishail] [~jbellis]  Is there already a separate bug for the second patch 
where the extend fails or should I create one?Or perhaps this one (7373) 
serves the purpose and 7275 fixes the CDL problem?



was (Author: jeffery.griffith):
[~mishail] [~jbellis]  Is there already a separate bug for the second patch 
where the extend fails or should I create one?Or perhaps this one (7373) 
serves the purpose and 7275 fixes the CDL problem.

 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
Assignee: Mikhail Stepura
 Fix For: 1.2.17, 2.0.9, 2.1.0

 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 
 0002-Handle-possible-integer-overflow.patch, 7373-v2.txt


 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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver0 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)
   

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-11 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/11/14 6:46 PM:
---

The NegativeArraySizeException seems to come from here. Seems like the 1.8G 
size was getting doubled resulting an an integer overflow:

private void expand(int i) {
/* Can the buffer handle @i more bytes, if not expand it */
if (count + i = buf.length) {
return;
}

byte[] newbuf = new byte[(count + i) * 2];
System.arraycopy(buf, 0, newbuf, 0, count);
buf = newbuf;
}


was (Author: jeffery.griffith):
The NegativeArraySize seems to come from here. Seems like the 1.8G size was 
getting doubled resulting an an integer overflow:

private void expand(int i) {
/* Can the buffer handle @i more bytes, if not expand it */
if (count + i = buf.length) {
return;
}

byte[] newbuf = new byte[(count + i) * 2];
System.arraycopy(buf, 0, newbuf, 0, count);
buf = newbuf;
}

 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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver0 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 
 

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-11 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/11/14 6:46 PM:
---

The NegativeArraySize seems to come from here. Seems like the 1.8G size was 
getting doubled resulting an an integer overflow:

private void expand(int i) {
/* Can the buffer handle @i more bytes, if not expand it */
if (count + i = buf.length) {
return;
}

byte[] newbuf = new byte[(count + i) * 2];
System.arraycopy(buf, 0, newbuf, 0, count);
buf = newbuf;
}


was (Author: jeffery.griffith):
The NegativeIndexException seems to come from here. Seems like the 1.8G size 
was getting doubled resulting an an integer overflow:

private void expand(int i) {
/* Can the buffer handle @i more bytes, if not expand it */
if (count + i = buf.length) {
return;
}

byte[] newbuf = new byte[(count + i) * 2];
System.arraycopy(buf, 0, newbuf, 0, count);
buf = newbuf;
}

 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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver0 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 
 

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-11 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/11/14 6:48 PM:
---

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. There is also a lot of 
GCing and Gossip thinks two nodes are down.

 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)

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-11 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/11/14 9:00 PM:
---

Thanks Mikhail. If my explanation for the exception above makes sense, it would 
seem the only mystery is how the high-traffic cf 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)



was (Author: jeffery.griffith):
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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver  

[jira] [Comment Edited] (CASSANDRA-7373) Commit logs no longer deleting and MemtablePostFlusher pending growing

2014-06-11 Thread Jeff Griffith (JIRA)

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

Jeff Griffith edited comment on CASSANDRA-7373 at 6/11/14 9:01 PM:
---

Thanks Mikhail. If my explanation for the exception above makes sense, it would 
seem the only mystery is how the high-traffic cf got to be over half of 
MAX-INT. In the logs below, is this amount about some crazy-large or corrupted 
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)



was (Author: jeffery.griffith):
Thanks Mikhail. If my explanation for the exception above makes sense, it would 
seem the only mystery is how the high-traffic cf 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 NameActive   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 02322201 0
  0
 ReplicateOnWriteStage 0 0  0 0
  0
 GossipStage   0 0 120780 0
  0
 AntiEntropyStage  0 0  0 0
  0
 MigrationStage0 0  0 0
  0
 MemoryMeter   0 0456 0
  0
 MemtablePostFlusher   1   447   6344 0
  0
 FlushWriter   0 0   6132 0
 62
 MiscStage 0 0  0 0
  0
 PendingRangeCalculator0 0  6 0
  0
 commitlog_archiver