[jira] [Commented] (CASSANDRA-11504) Slow inter-node network growth & gc issues with uptime

2016-04-05 Thread Francois Richard (JIRA)

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

Francois Richard commented on CASSANDRA-11504:
--

Please note that for specific nodes, we have enabled all the Cassandra metrics 
(via the CSVReporter) and at this point nothing seems to correlate with the 
increase of network traffic between nodes.

> Slow inter-node network growth & gc issues with uptime
> --
>
> Key: CASSANDRA-11504
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11504
> Project: Cassandra
>  Issue Type: Bug
> Environment: Cassandra 2.1.13
>Reporter: Jeff Griffith
> Attachments: InterNodeTraffic.jpg
>
>
> We are looking for help troubleshooting our production environment where we 
> are experiencing GC problems. After much experimentation and troubleshooting 
> with various settings, the only correlation that we can find with a slow 
> growth in GC is a slow growth in network traffic BETWEEN cassandra nodes in 
> our cluster. As an example, I have attached an example where in a cluster of 
> 24 nodes, i restarted 23 of them. Note that the outgoing rate for that 24th 
> node remains high while all others drop after the restart. Also note that 
> this graph is ONLY traffic between cassandra nodes. Traffic from the clients 
> remains FLAT throughout. Analyzing column family stats shows they are flat 
> throughout. Cache hit rates are also consistent across nodes. GC is of course 
> its own can of worms so we are hoping this considerable increase in traffic 
> (more than double over the course of 6rs) between nodes explains it. We would 
> greatly appreciate any ideas as to why this extra network output correlates 
> to uptime or ideas on what to "diff" between the nodes.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7368) Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException

2014-06-10 Thread Francois Richard (JIRA)

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

Francois Richard commented on CASSANDRA-7368:
-

The scrub of the sstables completed and the problems does not seem to happen 
again.  Coming back to the original problem, why did compaction totally stop on 
this node even if we have the following setting:

concurrent_compactors:  26.

Should there be more resilience at the compaction level to keep on going when 
some SSTABLE are corrupted? 


 Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException
 --

 Key: CASSANDRA-7368
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7368
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: OS: RHEL 6.5
 Cassandra version: 1.2.16
Reporter: Francois Richard

 Hi,
 We are getting a case where compaction stops totally on a node after an 
 exception related to: org.apache.cassandra.io.sstable.CorruptSSTableException.
 nodetool compactionstats remains at the same level for hours:
 {code}
 pending tasks: 1451
   compaction typekeyspace   column family   completed 
   total  unit  progress
CompactionSyncCoreContactPrefixBytesIndex   
 257799931   376785179 bytes68.42%
 Active compaction remaining time :n/a
 {code}
 Here is the exception log:
 {code}
 ERROR [Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db')]
  2014-06-09 06:39:37,570 CassandraDaemon.java (line 191) Exception in thread 
 Thread[Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db'),1,main]
 org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
 dataSize of 7421941880990663551 starting at 257836699 would be larger than 
 file 
 /home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db
  length 376785179
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:167)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:83)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:69)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:180)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:155)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:142)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:238)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:207)
   at 
 com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
   at 
 com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
 --
 {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7368) Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException

2014-06-09 Thread Francois Richard (JIRA)
Francois Richard created CASSANDRA-7368:
---

 Summary: Compaction stops after 
org.apache.cassandra.io.sstable.CorruptSSTableException
 Key: CASSANDRA-7368
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7368
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: OS: RHEL 6.5
Cassandra version: 1.2.16
Reporter: Francois Richard


Hi,

We are getting a case where compaction stops totally on a node after an 
exception related to: org.apache.cassandra.io.sstable.CorruptSSTableException.

nodetool compactionstats remains at the same level for hours:
{code}
pending tasks: 1451
  compaction typekeyspace   column family   completed   
total  unit  progress
   CompactionSyncCoreContactPrefixBytesIndex   
257799931   376785179 bytes68.42%
Active compaction remaining time :n/a
{code}


Here is the exception log:
{code}

ERROR [Deserialize 
SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db')]
 2014-06-09 06:39:37,570 CassandraDaemon.java (line 191) Exception in thread 
Thread[Deserialize 
SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db'),1,main]
org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
dataSize of 7421941880990663551 starting at 257836699 would be larger than file 
/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db
 length 376785179
at 
org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:167)
at 
org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:83)
at 
org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:69)
at 
org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:180)
at 
org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:155)
at 
org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:142)
at 
org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38)
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:238)
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:207)
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
--

{code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7368) Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException

2014-06-09 Thread Francois Richard (JIRA)

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

Francois Richard commented on CASSANDRA-7368:
-

Here are the cassandra.yaml settings related to compaction:

{code}
in_memory_compaction_limit_in_mb: 64
concurrent_compactors:  26
multithreaded_compaction: true
compaction_throughput_mb_per_sec: 0
compaction_preheat_key_cache: true
{code}

 Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException
 --

 Key: CASSANDRA-7368
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7368
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: OS: RHEL 6.5
 Cassandra version: 1.2.16
Reporter: Francois Richard

 Hi,
 We are getting a case where compaction stops totally on a node after an 
 exception related to: org.apache.cassandra.io.sstable.CorruptSSTableException.
 nodetool compactionstats remains at the same level for hours:
 {code}
 pending tasks: 1451
   compaction typekeyspace   column family   completed 
   total  unit  progress
CompactionSyncCoreContactPrefixBytesIndex   
 257799931   376785179 bytes68.42%
 Active compaction remaining time :n/a
 {code}
 Here is the exception log:
 {code}
 ERROR [Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db')]
  2014-06-09 06:39:37,570 CassandraDaemon.java (line 191) Exception in thread 
 Thread[Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db'),1,main]
 org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
 dataSize of 7421941880990663551 starting at 257836699 would be larger than 
 file 
 /home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db
  length 376785179
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:167)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:83)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:69)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:180)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:155)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:142)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:238)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:207)
   at 
 com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
   at 
 com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
 --
 {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7368) Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException

2014-06-09 Thread Francois Richard (JIRA)

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

Francois Richard commented on CASSANDRA-7368:
-

I have disabled multi-threaded compaction and now scrubbing the sstable in 
question. Will update soon.

 Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException
 --

 Key: CASSANDRA-7368
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7368
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: OS: RHEL 6.5
 Cassandra version: 1.2.16
Reporter: Francois Richard

 Hi,
 We are getting a case where compaction stops totally on a node after an 
 exception related to: org.apache.cassandra.io.sstable.CorruptSSTableException.
 nodetool compactionstats remains at the same level for hours:
 {code}
 pending tasks: 1451
   compaction typekeyspace   column family   completed 
   total  unit  progress
CompactionSyncCoreContactPrefixBytesIndex   
 257799931   376785179 bytes68.42%
 Active compaction remaining time :n/a
 {code}
 Here is the exception log:
 {code}
 ERROR [Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db')]
  2014-06-09 06:39:37,570 CassandraDaemon.java (line 191) Exception in thread 
 Thread[Deserialize 
 SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db'),1,main]
 org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
 dataSize of 7421941880990663551 starting at 257836699 would be larger than 
 file 
 /home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db
  length 376785179
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:167)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:83)
   at 
 org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:69)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:180)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:155)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:142)
   at 
 org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:238)
   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:207)
   at 
 com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
   at 
 com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
 --
 {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


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

2014-06-09 Thread Francois Richard (JIRA)
Francois Richard created CASSANDRA-7373:
---

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