You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Anton Winter <an...@myrddin.org> on 2011/08/11 17:16:36 UTC

performance problems on new cluster

Hi,

I have recently been migrating to a small 12 node Cassandra cluster 
spanning across 4 DC's and have been encountering various issues with 
what I suspect to be a performance tuning issue with my data set.  I've 
learnt a few lessons along the way but I'm at a bit of a roadblock now 
where I have been experiencing frequent OutOfMemory exceptions, various 
other exceptions, poor performance and my ring is appearing to become 
imbalanced during repairs.  I've tried various different configurations 
but haven't been able to get to the bottom of my performance issues.  
I'm assuming this has something to do with my data and some performance 
tuning metric that I'm merely overlooking.

My ring was created as documented in the wiki & various other 
performance tuning guides, calculating the tokens at each DC and 
incrementing when in conflict.  It is as follows:

Address         DC          Rack        Status State   Load            
Owns    Token
                                                                                113427455640312821154458202477256070487
dc1host1  dc1          1a          Up     Normal  88.62 GB        33.33%  0
dc2host1  dc2          1           Up     Normal  14.76 GB        0.00%   1
dc3host1    dc3          1           Up     Normal  15.99 GB        
0.00%   2
dc4host1    cd4          1           Up     Normal  14.52 GB        
0.00%   3
dc1host2   dc1          1a          Up     Normal  18.02 GB        
33.33%  56713727820156410577229101238628035242
dc2host2  dc2          1           Up     Normal  16.5 GB         
0.00%   56713727820156410577229101238628035243
dc3host2     dc3          1           Up     Normal  16.37 GB        
0.00%   56713727820156410577229101238628035244
dc4host2    dc4          1           Up     Normal  13.34 GB        
0.00%   56713727820156410577229101238628035245
dc1host3  dc1          1a          Up     Normal  16.59 GB        
33.33%  113427455640312821154458202477256070484
dc2host3   dc2          1           Up     Normal  15.22 GB        
0.00%   113427455640312821154458202477256070485
dc3host3   dc3          1           Up     Normal  15.59 GB        
0.00%   113427455640312821154458202477256070486
dc4host3    dc4          1           Up     Normal  8.84 GB         
0.00%   113427455640312821154458202477256070487

The above ring was freshly created and fairly evenly distributed in load 
prior to a repair (which is still running at the time of the above 
command) on dc1host1, however with the exception of dc4host3 where a 
previous bulk data load timed out.  dc4host3 was responding poorly, was 
failing according to other nodes and judging from its heap usage was 
rather close to OOM'ing before it was restarted.

I'm also using NTS with RF2.

The primary issues I'm experiencing are:

Light load against nodes in dc1 was causing OutOfMemory exceptions 
across all Cassandra servers outside of dc1 which were all idle and 
eventually after several hours happened on one of the dc1 nodes.  This 
issue was produced using svn trunk r1153002 and an in house written 
Snitched which effectively combined PropertyFileSnitch with some 
components of Ec2Snitch.  While trying to resolve these issues I have 
moved to a r1156490 snapshot and have switched across to just the 
PropertyFileSnitch and simply utilising the broadcast_address 
configuration option available in trunk which seems to work quite well.

Since moving to r1156490 we have stopped getting OOM's, but that may 
actually be because we have been unable to send traffic to the cluster 
to be able to produce one.

The most current issues I have been experiencing are the following:

1) thrift timeouts & general degraded response times
2) *lots* of exception errors, such as:

ERROR [ReadRepairStage:1076] 2011-08-11 13:33:41,266 
AbstractCassandraDaemon.java (line 133) Fatal exception in thread 
Thread[ReadRepairStage:1076,5,main]
java.lang.AssertionError
         at 
org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:73)
         at 
org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
         at java.lang.Thread.run(Thread.java:662)

3) ring imbalances during a repair (refer to the above nodetool ring output)
4) regular failure detection when any node does something only 
moderately stressful, such as a repair or are under light load etc. but 
the node itself thinks it is fine.

My hosts are all 32Gb with either 4 or 16 cores, I've set heaps 
appropriately to half physical memory (16G) and for the purpose of 
cluster simplicity set all younggen to 400Mb.  JNA is in use, commitlogs 
and data have been split onto different filesystems and so on.

My data set as described by a dev is essentially as follows:

3 column families (tables):

cf1.  The RowKey is the user id.  This is the primary column family 
queried on and always just looked up by RowKey.  It has 1 supercolumn 
called "seg".  The column names in this supercolumn are the segment_id's 
that the user belongs to and the value is just "1".  This should have 
about 150mm rows.  Each row will have an average of 2-3 columns in the 
"seg" supercolumn.  The column values have TTL's set on them.

cf2.  This is a CounterColumnFamily.  There's only a single "cnt" column 
which stores a counter of the number of cf1's having that segment.  This 
was only updated during the import and is not read at all.

cf3.  This is a lookup between the RowKey which is an external ID and 
the RowKey to be used to find the user in the cf1  CF.


Does anyone have any ideas or suggestions about where I should be 
focusing on to get to the bottom of these issues or any recommendations 
on where I should be focusing my efforts on?

Thanks,
Anton





Re: performance problems on new cluster

Posted by Raj N <ra...@gmail.com>.
How did you solve your problem eventually? I am experiencing something
similar. Did you run cleanup on the node that has 80GB data?

-Raj

On Mon, Aug 15, 2011 at 10:12 PM, aaron morton <aa...@thelastpickle.com>wrote:

> Just checking do you have read_repair_chance set to something ? The second
> request is going to all replicas which should only happen with CL ONE if
> read repair is running for the request.
>
> The exceptions are happening during read repair which is running async to
> the main read request. It's occurring after we have detected a digest mis
> match, when the process is trying to reconcile the full data responses from
> the replicas. The Assertion error is happening because the replica sent a
> digest response. The NPE is probably happening because the response did not
> include a row, how / why the response is not marked as digest is a mystery.
>
> This may be related to the main problem. If not dont forget to some back
> to it.
>
> In you first log with the timeout something is not right…
> > DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,187 StorageProxy.java (line
> 546) reading data from /dc1host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:26:35,191 StorageProxy.java (line
> 593) Read timeout: java.util.concurrent.TimeoutException: Operation timed
> out - received only 1 responses from /dc1host3,  .
> The reading… log messages are written before the inter node messages are
> sent. For this CL ONE read only node dc 1 host 3 is involved and it has
> been asked for the data response. Makes sense if Read Repair is not running
> for the request.
>
> *But* the timeout error says we got a response from dc 1 host 3. One way I
> can see that happening is dc 1 host 3 returning a digest instead of a data
> response (see o.a.c.service.ReadCallback.response(Message)). Which kind of
> matches what we saw above.
>
> We need some more extensive logging and probably a trip to
> https://issues.apache.org/jira/browse/CASSANDRA
>
> Would be good to see full DEBUG logs from both dc1 host 1 and dc1 host 3
> if you can that reproduce the fault like the first one. Turn off read
> repair to make things a little simpler. If thats too much we need
> StorageProxy, ReadCalback, ReadVerbHandler
>
> Can you update the email thread with the ticket.
>
> Thanks
> A
>
> -----------------
> Aaron Morton
> Freelance Cassandra Developer
> @aaronmorton
> http://www.thelastpickle.com
>
> On 15/08/2011, at 7:34 PM, Anton Winter wrote:
>
> >> OK, node latency is fine and you are using some pretty low
> >> consistency. You said NTS with RF 2, is that RF 2 for each DC ?
> >
> > Correct, I'm using RF 2 for each DC.
> >
> >
> >
> > I was able to reproduce the cli timeouts on the non replica nodes.
> >
> > The debug log output from dc1host1 (non replica node):
> >
> > DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,183 StorageProxy.java (line
> 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1',
> key='userid1', column_parent='QueryPath(columnFamilyName='cf1',
> superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]',
> columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> > DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,187 StorageProxy.java (line
> 546) reading data from /dc1host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:26:35,191 StorageProxy.java (line
> 593) Read timeout: java.util.concurrent.TimeoutException: Operation timed
> out - received only 1 responses from /dc1host3,  .
> >
> >
> > If the query is run again on the same node (dc1host1) 0 rows are
> returned and the following DEBUG messages are logged:
> >
> >
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line
> 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1',
> key='userid1', column_parent='QueryPath(columnFamilyName='cf1',
> superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]',
> columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line
> 546) reading data from /dc1host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line
> 562) reading digest from /dc1host2
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc2host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc2host2
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc3host2
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc3host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc4host3
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line
> 562) reading digest from /dc4host2
> > DEBUG [pool-2-thread-14] 2011-08-15 05:32:06,022 StorageProxy.java (line
> 588) Read: 508 ms.
> > ERROR [ReadRepairStage:2112] 2011-08-15 05:32:06,404
> AbstractCassandraDaemon.java (line 133) Fatal exception in thread
> Thread[ReadRepairStage:2112,5,main]
> > java.lang.AssertionError
> >        at
> org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:73)
> >        at
> org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
> >        at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> >        at java.lang.Thread.run(Thread.java:662)
> >
> >
> > Subsequent queries afterwards show "reading data from /dc1host2" however
> the results returned remains 0.
> >
> >
> > If I run the same query on a replica I get the correct result returned
> but with 2 exceptions as follows:
> >
> >
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,792 StorageProxy.java (line
> 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1',
> key='userid1', column_parent='QueryPath(columnFamilyName='cf1',
> superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]',
> columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line
> 541) reading data locally
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line
> 562) reading digest from /dc1host3
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line
> 562) reading digest from dns.entry.for.dc3host2/dc3host2
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line
> 562) reading digest from dns.entry.for.dc3host3/dc3host3
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line
> 562) reading digest from dns.entry.for.dc2host2/dc2host2
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line
> 562) reading digest from dns.entry.for.dc2host3/dc2host3
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line
> 562) reading digest from dc4host2/dc4host2
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line
> 562) reading digest from dc4host3/dc4host3
> > DEBUG [ReadStage:20102] 2011-08-15 05:45:49,793 StorageProxy.java (line
> 690) LocalReadRunnable reading SliceFromReadCommand(table='ks1',
> key='userid1', column_parent='QueryPath(columnFamilyName='cf1',
> superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]',
> columnName='null')', start='', finish='', reversed=false, count=1000000)
> > DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,795 StorageProxy.java (line
> 588) Read: 1 ms.
> > ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,890
> AbstractCassandraDaemon.java (line 133) Fatal exception in thread
> Thread[ReadRepairStage:11,5,main]
> > java.lang.RuntimeException: java.lang.NullPointerException
> >        at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> >        at java.lang.Thread.run(Thread.java:662)
> > Caused by: java.lang.NullPointerException
> >        at
> org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
> >        at
> org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
> >        at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> >        ... 3 more
> > ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,892
> AbstractCassandraDaemon.java (line 133) Fatal exception in thread
> Thread[ReadRepairStage:11,5,main]
> > java.lang.RuntimeException: java.lang.NullPointerException
> >        at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> >        at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> >        at java.lang.Thread.run(Thread.java:662)
> > Caused by: java.lang.NullPointerException
> >        at
> org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
> >        at
> org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
> >        at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> >        ... 3 more
> >
> > The above have been reproduced in each DC.
> >
> >
> > The testing that has been performing poorly happens to be on the non
> replica nodes.  Initial testing against one of the replicas appears that
> they do not exhibit the performance problems that the non replicas do.
> >
> > Regards,
> > Anton
> >
> >
> >
> > On Fri, 12 Aug 2011 14:11:06 +1200, aaron morton wrote:
> >>>
> >>> iostat doesn't show a request queue bottleneck.  The timeouts we are
> seeing is for reads.  The latency on the nodes I have temporarily used for
> reads is around 2-45ms.  The next token in the ring at an alternate DC is
> showing ~4ms with everything else around 0.05ms.  tpstats desn't show any
> active/pending.  Reads are at CL.ONE & Writes using CL.ANY
> >>
> >> OK, node latency is fine and you are using some pretty low
> >> consistency. You said NTS with RF 2, is that RF 2 for each DC ?
> >>
> >> The steps below may help get an idea of whats going on…
> >>
> >> 1) use nodetool getendpoints to determine which replicas a key is.
> >> 2) connect directly to one of the endpoints with the CLI, ensure CL
> >> is ONE and do your test query.
> >> 3) connect to another node in the same DC that is not a replica and
> >> do the same.
> >> 4) connect to another node in a different DC and do the same
> >>
> >> Once you can repo it try turning up the logging not the coordinator
> >> to DEBUG you can do this via JConsole. Look for these lines….
> >>
> >> * Command/ConsistencyLevel is….
> >> * reading data locally... or reading data from…
> >> * reading digest locally… or reading digest for from…
> >> * Read timeout:…
> >>
> >> You'll also see some lines about receiving messages from other nodes.
> >>
> >> Hopefully you can get an idea of which nodes are involved in a
> >> failing query. Getting a thrift TimedOutException on a read with CL
> >> ONE is pretty odd.
> >>
> >>> What can I do in regards to confirming this issue is still outstanding
> and/or we are affected by it?
> >> It's in 0.8 and will not be fixed. My unscientific approach was to
> >> repair a single CF at a time, hoping that the differences would be
> >> smaller and less data would be streamed.
> >> Minor compaction should help squish things down. If you want to get
> >> more aggressive reduce the min compaction threshold and trigger a
> >> minor compaction with nodetool flush.
> >>
> >>> Version of failure detection?  I've not seen anything on this so I
> suspect this is the default.
> >> Was asking so I could see if their were any fixed in Gossip or the
> >> FailureDetect that you were missing. Check the CHANGES.txt file.
> >>
> >> Hope that helps.
> >
>
>

Re: performance problems on new cluster

Posted by aaron morton <aa...@thelastpickle.com>.
Just checking do you have read_repair_chance set to something ? The second request is going to all replicas which should only happen with CL ONE if read repair is running for the request. 

The exceptions are happening during read repair which is running async to the main read request. It's occurring after we have detected a digest mis match, when the process is trying to reconcile the full data responses from the replicas. The Assertion error is happening because the replica sent a digest response. The NPE is probably happening because the response did not include a row, how / why the response is not marked as digest is a mystery. 

This may be related to the main problem. If not dont forget to some back to it.

In you first log with the timeout something is not right…
> DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,187 StorageProxy.java (line 546) reading data from /dc1host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:26:35,191 StorageProxy.java (line 593) Read timeout: java.util.concurrent.TimeoutException: Operation timed out - received only 1 responses from /dc1host3,  .
The reading… log messages are written before the inter node messages are sent. For this CL ONE read only node dc 1 host 3 is involved and it has been asked for the data response. Makes sense if Read Repair is not running for the request. 

*But* the timeout error says we got a response from dc 1 host 3. One way I can see that happening is dc 1 host 3 returning a digest instead of a data response (see o.a.c.service.ReadCallback.response(Message)). Which kind of matches what we saw above. 

We need some more extensive logging and probably a trip to https://issues.apache.org/jira/browse/CASSANDRA

Would be good to see full DEBUG logs from both dc1 host 1 and dc1 host 3 if you can that reproduce the fault like the first one. Turn off read repair to make things a little simpler. If thats too much we need StorageProxy, ReadCalback, ReadVerbHandler

Can you update the email thread with the ticket. 

Thanks
A

-----------------
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com

On 15/08/2011, at 7:34 PM, Anton Winter wrote:

>> OK, node latency is fine and you are using some pretty low
>> consistency. You said NTS with RF 2, is that RF 2 for each DC ?
> 
> Correct, I'm using RF 2 for each DC.
> 
> 
> 
> I was able to reproduce the cli timeouts on the non replica nodes.
> 
> The debug log output from dc1host1 (non replica node):
> 
> DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,183 StorageProxy.java (line 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', key='userid1', column_parent='QueryPath(columnFamilyName='cf1', superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,187 StorageProxy.java (line 546) reading data from /dc1host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:26:35,191 StorageProxy.java (line 593) Read timeout: java.util.concurrent.TimeoutException: Operation timed out - received only 1 responses from /dc1host3,  .
> 
> 
> If the query is run again on the same node (dc1host1) 0 rows are returned and the following DEBUG messages are logged:
> 
> 
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', key='userid1', column_parent='QueryPath(columnFamilyName='cf1', superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line 546) reading data from /dc1host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java (line 562) reading digest from /dc1host2
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc2host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc2host2
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc3host2
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc3host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc4host3
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java (line 562) reading digest from /dc4host2
> DEBUG [pool-2-thread-14] 2011-08-15 05:32:06,022 StorageProxy.java (line 588) Read: 508 ms.
> ERROR [ReadRepairStage:2112] 2011-08-15 05:32:06,404 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadRepairStage:2112,5,main]
> java.lang.AssertionError
>        at org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:73)
>        at org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> 
> 
> Subsequent queries afterwards show "reading data from /dc1host2" however the results returned remains 0.
> 
> 
> If I run the same query on a replica I get the correct result returned but with 2 exceptions as follows:
> 
> 
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,792 StorageProxy.java (line 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', key='userid1', column_parent='QueryPath(columnFamilyName='cf1', superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', columnName='null')', start='', finish='', reversed=false, count=1000000)/ONE
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 541) reading data locally
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 562) reading digest from /dc1host3
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 562) reading digest from dns.entry.for.dc3host2/dc3host2
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 562) reading digest from dns.entry.for.dc3host3/dc3host3
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 562) reading digest from dns.entry.for.dc2host2/dc2host2
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 562) reading digest from dns.entry.for.dc2host3/dc2host3
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 562) reading digest from dc4host2/dc4host2
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 562) reading digest from dc4host3/dc4host3
> DEBUG [ReadStage:20102] 2011-08-15 05:45:49,793 StorageProxy.java (line 690) LocalReadRunnable reading SliceFromReadCommand(table='ks1', key='userid1', column_parent='QueryPath(columnFamilyName='cf1', superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', columnName='null')', start='', finish='', reversed=false, count=1000000)
> DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,795 StorageProxy.java (line 588) Read: 1 ms.
> ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,890 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadRepairStage:11,5,main]
> java.lang.RuntimeException: java.lang.NullPointerException
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> Caused by: java.lang.NullPointerException
>        at org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
>        at org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>        ... 3 more
> ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,892 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadRepairStage:11,5,main]
> java.lang.RuntimeException: java.lang.NullPointerException
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> Caused by: java.lang.NullPointerException
>        at org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
>        at org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>        ... 3 more
> 
> The above have been reproduced in each DC.
> 
> 
> The testing that has been performing poorly happens to be on the non replica nodes.  Initial testing against one of the replicas appears that they do not exhibit the performance problems that the non replicas do.
> 
> Regards,
> Anton
> 
> 
> 
> On Fri, 12 Aug 2011 14:11:06 +1200, aaron morton wrote:
>>> 
>>> iostat doesn't show a request queue bottleneck.  The timeouts we are seeing is for reads.  The latency on the nodes I have temporarily used for reads is around 2-45ms.  The next token in the ring at an alternate DC is showing ~4ms with everything else around 0.05ms.  tpstats desn't show any active/pending.  Reads are at CL.ONE & Writes using CL.ANY
>> 
>> OK, node latency is fine and you are using some pretty low
>> consistency. You said NTS with RF 2, is that RF 2 for each DC ?
>> 
>> The steps below may help get an idea of whats going on…
>> 
>> 1) use nodetool getendpoints to determine which replicas a key is.
>> 2) connect directly to one of the endpoints with the CLI, ensure CL
>> is ONE and do your test query.
>> 3) connect to another node in the same DC that is not a replica and
>> do the same.
>> 4) connect to another node in a different DC and do the same
>> 
>> Once you can repo it try turning up the logging not the coordinator
>> to DEBUG you can do this via JConsole. Look for these lines….
>> 
>> * Command/ConsistencyLevel is….
>> * reading data locally... or reading data from…
>> * reading digest locally… or reading digest for from…
>> * Read timeout:…
>> 
>> You'll also see some lines about receiving messages from other nodes.
>> 
>> Hopefully you can get an idea of which nodes are involved in a
>> failing query. Getting a thrift TimedOutException on a read with CL
>> ONE is pretty odd.
>> 
>>> What can I do in regards to confirming this issue is still outstanding and/or we are affected by it?
>> It's in 0.8 and will not be fixed. My unscientific approach was to
>> repair a single CF at a time, hoping that the differences would be
>> smaller and less data would be streamed.
>> Minor compaction should help squish things down. If you want to get
>> more aggressive reduce the min compaction threshold and trigger a
>> minor compaction with nodetool flush.
>> 
>>> Version of failure detection?  I've not seen anything on this so I suspect this is the default.
>> Was asking so I could see if their were any fixed in Gossip or the
>> FailureDetect that you were missing. Check the CHANGES.txt file.
>> 
>> Hope that helps.
> 


Re: performance problems on new cluster

Posted by Anton Winter <an...@myrddin.org>.
> OK, node latency is fine and you are using some pretty low
> consistency. You said NTS with RF 2, is that RF 2 for each DC ?

Correct, I'm using RF 2 for each DC.



I was able to reproduce the cli timeouts on the non replica nodes.

The debug log output from dc1host1 (non replica node):

DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,183 StorageProxy.java 
(line 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', 
key='userid1', column_parent='QueryPath(columnFamilyName='cf1', 
superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', 
columnName='null')', start='', finish='', reversed=false, 
count=1000000)/ONE
DEBUG [pool-2-thread-14] 2011-08-15 05:26:15,187 StorageProxy.java 
(line 546) reading data from /dc1host3
DEBUG [pool-2-thread-14] 2011-08-15 05:26:35,191 StorageProxy.java 
(line 593) Read timeout: java.util.concurrent.TimeoutException: 
Operation timed out - received only 1 responses from /dc1host3,  .


If the query is run again on the same node (dc1host1) 0 rows are 
returned and the following DEBUG messages are logged:


DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java 
(line 518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', 
key='userid1', column_parent='QueryPath(columnFamilyName='cf1', 
superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', 
columnName='null')', start='', finish='', reversed=false, 
count=1000000)/ONE
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java 
(line 546) reading data from /dc1host3
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,513 StorageProxy.java 
(line 562) reading digest from /dc1host2
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc2host3
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc2host2
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc3host2
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc3host3
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc4host3
DEBUG [pool-2-thread-14] 2011-08-15 05:32:05,514 StorageProxy.java 
(line 562) reading digest from /dc4host2
DEBUG [pool-2-thread-14] 2011-08-15 05:32:06,022 StorageProxy.java 
(line 588) Read: 508 ms.
ERROR [ReadRepairStage:2112] 2011-08-15 05:32:06,404 
AbstractCassandraDaemon.java (line 133) Fatal exception in thread 
Thread[ReadRepairStage:2112,5,main]
java.lang.AssertionError
         at 
org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:73)
         at 
org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
         at java.lang.Thread.run(Thread.java:662)


Subsequent queries afterwards show "reading data from /dc1host2" 
however the results returned remains 0.


If I run the same query on a replica I get the correct result returned 
but with 2 exceptions as follows:


DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,792 StorageProxy.java (line 
518) Command/ConsistencyLevel is SliceFromReadCommand(table='ks1', 
key='userid1', column_parent='QueryPath(columnFamilyName='cf1', 
superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', 
columnName='null')', start='', finish='', reversed=false, 
count=1000000)/ONE
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 
541) reading data locally
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 
562) reading digest from /dc1host3
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 
562) reading digest from dns.entry.for.dc3host2/dc3host2
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,793 StorageProxy.java (line 
562) reading digest from dns.entry.for.dc3host3/dc3host3
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 
562) reading digest from dns.entry.for.dc2host2/dc2host2
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 
562) reading digest from dns.entry.for.dc2host3/dc2host3
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 
562) reading digest from dc4host2/dc4host2
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,794 StorageProxy.java (line 
562) reading digest from dc4host3/dc4host3
DEBUG [ReadStage:20102] 2011-08-15 05:45:49,793 StorageProxy.java (line 
690) LocalReadRunnable reading SliceFromReadCommand(table='ks1', 
key='userid1', column_parent='QueryPath(columnFamilyName='cf1', 
superColumnName='java.nio.HeapByteBuffer[pos=64 lim=67 cap=109]', 
columnName='null')', start='', finish='', reversed=false, count=1000000)
DEBUG [pool-2-thread-5] 2011-08-15 05:45:49,795 StorageProxy.java (line 
588) Read: 1 ms.
ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,890 
AbstractCassandraDaemon.java (line 133) Fatal exception in thread 
Thread[ReadRepairStage:11,5,main]
java.lang.RuntimeException: java.lang.NullPointerException
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
         at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.NullPointerException
         at 
org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
         at 
org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
         ... 3 more
ERROR [ReadRepairStage:11] 2011-08-15 05:45:50,892 
AbstractCassandraDaemon.java (line 133) Fatal exception in thread 
Thread[ReadRepairStage:11,5,main]
java.lang.RuntimeException: java.lang.NullPointerException
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
         at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
         at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.NullPointerException
         at 
org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:74)
         at 
org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
         at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
         ... 3 more

The above have been reproduced in each DC.


The testing that has been performing poorly happens to be on the non 
replica nodes.  Initial testing against one of the replicas appears that 
they do not exhibit the performance problems that the non replicas do.

Regards,
Anton



On Fri, 12 Aug 2011 14:11:06 +1200, aaron morton wrote:
>>
>> iostat doesn't show a request queue bottleneck.  The timeouts we are 
>> seeing is for reads.  The latency on the nodes I have temporarily used 
>> for reads is around 2-45ms.  The next token in the ring at an 
>> alternate DC is showing ~4ms with everything else around 0.05ms.  
>> tpstats desn't show any active/pending.  Reads are at CL.ONE & Writes 
>> using CL.ANY
>
> OK, node latency is fine and you are using some pretty low
> consistency. You said NTS with RF 2, is that RF 2 for each DC ?
>
> The steps below may help get an idea of whats going on…
>
> 1) use nodetool getendpoints to determine which replicas a key is.
> 2) connect directly to one of the endpoints with the CLI, ensure CL
> is ONE and do your test query.
> 3) connect to another node in the same DC that is not a replica and
> do the same.
> 4) connect to another node in a different DC and do the same
>
> Once you can repo it try turning up the logging not the coordinator
> to DEBUG you can do this via JConsole. Look for these lines….
>
> * Command/ConsistencyLevel is….
> * reading data locally... or reading data from…
> * reading digest locally… or reading digest for from…
> * Read timeout:…
>
> You'll also see some lines about receiving messages from other nodes.
>
> Hopefully you can get an idea of which nodes are involved in a
> failing query. Getting a thrift TimedOutException on a read with CL
> ONE is pretty odd.
>
>> What can I do in regards to confirming this issue is still 
>> outstanding and/or we are affected by it?
> It's in 0.8 and will not be fixed. My unscientific approach was to
> repair a single CF at a time, hoping that the differences would be
> smaller and less data would be streamed.
> Minor compaction should help squish things down. If you want to get
> more aggressive reduce the min compaction threshold and trigger a
> minor compaction with nodetool flush.
>
>> Version of failure detection?  I've not seen anything on this so I 
>> suspect this is the default.
> Was asking so I could see if their were any fixed in Gossip or the
> FailureDetect that you were missing. Check the CHANGES.txt file.
>
> Hope that helps.


Re: performance problems on new cluster

Posted by aaron morton <aa...@thelastpickle.com>.
> 
> iostat doesn't show a request queue bottleneck.  The timeouts we are seeing is for reads.  The latency on the nodes I have temporarily used for reads is around 2-45ms.  The next token in the ring at an alternate DC is showing ~4ms with everything else around 0.05ms.  tpstats desn't show any active/pending.  Reads are at CL.ONE & Writes using CL.ANY

OK, node latency is fine and you are using some pretty low consistency. You said NTS with RF 2, is that RF 2 for each DC ? 

The steps below may help get an idea of whats going on…

1) use nodetool getendpoints to determine which replicas a key is.  
2) connect directly to one of the endpoints with the CLI, ensure CL is ONE and do your test query. 
3) connect to another node in the same DC that is not a replica and do the same. 
4) connect to another node in a different DC and do the same 

Once you can repo it try turning up the logging not the coordinator to DEBUG you can do this via JConsole. Look for these lines….

* Command/ConsistencyLevel is….
* reading data locally... or reading data from…
* reading digest locally… or reading digest for from…
* Read timeout:…

You'll also see some lines about receiving messages from other nodes.  

Hopefully you can get an idea of which nodes are involved in a failing query. Getting a thrift TimedOutException on a read with CL ONE is pretty odd. 

> What can I do in regards to confirming this issue is still outstanding and/or we are affected by it?
It's in 0.8 and will not be fixed. My unscientific approach was to repair a single CF at a time, hoping that the differences would be smaller and less data would be streamed. 
Minor compaction should help squish things down. If you want to get more aggressive reduce the min compaction threshold and trigger a minor compaction with nodetool flush.   

> Version of failure detection?  I've not seen anything on this so I suspect this is the default.
Was asking so I could see if their were any fixed in Gossip or the FailureDetect that you were missing. Check the CHANGES.txt file. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com

On 12 Aug 2011, at 12:48, Anton Winter wrote:

> 
>> Is there a reason you are using the trunk and not one of the tagged releases? Official releases are a lot more stable than the trunk.
>> 
> Yes, as we are using a combination of Ec2 and colo servers we are needing to use broadcast_address from CASSANDRA-2491.  The patch that is associated with that JIRA does not apply cleanly against 0.8 so this is why we are using trunk.
> 
>>> 1) thrift timeouts & general degraded response times
>> For read or writes ? What sort of queries are you running ? Check the local latency on each node using cfstats and cfhistogram, and a bit of iostat http://spyced.blogspot.com/2010/01/linux-performance-basics.html What does nodetool tpstats say, is there a stage backing up?
>> 
>> If the local latency is OK look at the cross DC situation. What CL are you using? Are nodes timing out waiting for nodes in other DC's ?
> 
> iostat doesn't show a request queue bottleneck.  The timeouts we are seeing is for reads.  The latency on the nodes I have temporarily used for reads is around 2-45ms.  The next token in the ring at an alternate DC is showing ~4ms with everything else around 0.05ms.  tpstats desn't show any active/pending.  Reads are at CL.ONE & Writes using CL.ANY
> 
>> 
>>> 2) *lots* of exception errors, such as:
>> Repair is trying to run on a response which is a digest response, this should not be happening. Can you provide some more info on the type of query you are running ?
>> 
> The query being run is  get cf1['user-id']['seg']
> 
> 
>>> 3) ring imbalances during a repair (refer to the above nodetool ring output)
>> You may be seeing this
>> https://issues.apache.org/jira/browse/CASSANDRA-2280
>> I think it's a mistake that is it marked as resolved.
>> 
> What can I do in regards to confirming this issue is still outstanding and/or we are affected by it?
> 
>>> 4) regular failure detection when any node does something only moderately stressful, such as a repair or are under light load etc. but the node itself thinks it is fine.
>> What version are you using ?
>> 
> Version of failure detection?  I've not seen anything on this so I suspect this is the default.
> 
> 
> Thanks,
> Anton
> 


Re: performance problems on new cluster

Posted by Anton Winter <an...@myrddin.org>.
> Is there a reason you are using the trunk and not one of the tagged 
> releases? Official releases are a lot more stable than the trunk.
>
Yes, as we are using a combination of Ec2 and colo servers we are 
needing to use broadcast_address from CASSANDRA-2491.  The patch that is 
associated with that JIRA does not apply cleanly against 0.8 so this is 
why we are using trunk.

>> 1) thrift timeouts & general degraded response times
> For read or writes ? What sort of queries are you running ? Check the 
> local latency on each node using cfstats and cfhistogram, and a bit of 
> iostat 
> http://spyced.blogspot.com/2010/01/linux-performance-basics.html What 
> does nodetool tpstats say, is there a stage backing up?
>
> If the local latency is OK look at the cross DC situation. What CL are 
> you using? Are nodes timing out waiting for nodes in other DC's ?

iostat doesn't show a request queue bottleneck.  The timeouts we are 
seeing is for reads.  The latency on the nodes I have temporarily used 
for reads is around 2-45ms.  The next token in the ring at an alternate 
DC is showing ~4ms with everything else around 0.05ms.  tpstats desn't 
show any active/pending.  Reads are at CL.ONE & Writes using CL.ANY

>
>> 2) *lots* of exception errors, such as:
> Repair is trying to run on a response which is a digest response, this 
> should not be happening. Can you provide some more info on the type of 
> query you are running ?
>
The query being run is  get cf1['user-id']['seg']


>> 3) ring imbalances during a repair (refer to the above nodetool ring 
>> output)
> You may be seeing this
> https://issues.apache.org/jira/browse/CASSANDRA-2280
> I think it's a mistake that is it marked as resolved.
>
What can I do in regards to confirming this issue is still outstanding 
and/or we are affected by it?

>> 4) regular failure detection when any node does something only 
>> moderately stressful, such as a repair or are under light load etc. 
>> but the node itself thinks it is fine.
> What version are you using ?
>
Version of failure detection?  I've not seen anything on this so I 
suspect this is the default.


Thanks,
Anton


Re: performance problems on new cluster

Posted by aaron morton <aa...@thelastpickle.com>.
Is there a reason you are using the trunk and not one of the tagged releases? Official releases are a lot more stable than the trunk. 

> 1) thrift timeouts & general degraded response times
For read or writes ? What sort of queries are you running ? Check the local latency on each node using cfstats and cfhistogram, and a bit of iostat http://spyced.blogspot.com/2010/01/linux-performance-basics.html What does nodetool tpstats say, is there a stage backing up?

If the local latency is OK look at the cross DC situation. What CL are you using? Are nodes timing out waiting for nodes in other DC's ? 

> 2) *lots* of exception errors, such as:
Repair is trying to run on a response which is a digest response, this should not be happening. Can you provide some more info on the type of query you are running ? 

> 3) ring imbalances during a repair (refer to the above nodetool ring output)
You may be seeing this
https://issues.apache.org/jira/browse/CASSANDRA-2280
I think it's a mistake that is it marked as resolved. 

> 4) regular failure detection when any node does something only moderately stressful, such as a repair or are under light load etc. but the node itself thinks it is fine.
What version are you using ? 

I'd take a look at the exceptions first then move onto the performance issues. 

Cheers

-----------------
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com

On 12 Aug 2011, at 03:16, Anton Winter wrote:

> Hi,
> 
> I have recently been migrating to a small 12 node Cassandra cluster spanning across 4 DC's and have been encountering various issues with what I suspect to be a performance tuning issue with my data set.  I've learnt a few lessons along the way but I'm at a bit of a roadblock now where I have been experiencing frequent OutOfMemory exceptions, various other exceptions, poor performance and my ring is appearing to become imbalanced during repairs.  I've tried various different configurations but haven't been able to get to the bottom of my performance issues.  I'm assuming this has something to do with my data and some performance tuning metric that I'm merely overlooking.
> 
> My ring was created as documented in the wiki & various other performance tuning guides, calculating the tokens at each DC and incrementing when in conflict.  It is as follows:
> 
> Address         DC          Rack        Status State   Load            Owns    Token
>                                                                               113427455640312821154458202477256070487
> dc1host1  dc1          1a          Up     Normal  88.62 GB        33.33%  0
> dc2host1  dc2          1           Up     Normal  14.76 GB        0.00%   1
> dc3host1    dc3          1           Up     Normal  15.99 GB        0.00%   2
> dc4host1    cd4          1           Up     Normal  14.52 GB        0.00%   3
> dc1host2   dc1          1a          Up     Normal  18.02 GB        33.33%  56713727820156410577229101238628035242
> dc2host2  dc2          1           Up     Normal  16.5 GB         0.00%   56713727820156410577229101238628035243
> dc3host2     dc3          1           Up     Normal  16.37 GB        0.00%   56713727820156410577229101238628035244
> dc4host2    dc4          1           Up     Normal  13.34 GB        0.00%   56713727820156410577229101238628035245
> dc1host3  dc1          1a          Up     Normal  16.59 GB        33.33%  113427455640312821154458202477256070484
> dc2host3   dc2          1           Up     Normal  15.22 GB        0.00%   113427455640312821154458202477256070485
> dc3host3   dc3          1           Up     Normal  15.59 GB        0.00%   113427455640312821154458202477256070486
> dc4host3    dc4          1           Up     Normal  8.84 GB         0.00%   113427455640312821154458202477256070487
> 
> The above ring was freshly created and fairly evenly distributed in load prior to a repair (which is still running at the time of the above command) on dc1host1, however with the exception of dc4host3 where a previous bulk data load timed out.  dc4host3 was responding poorly, was failing according to other nodes and judging from its heap usage was rather close to OOM'ing before it was restarted.
> 
> I'm also using NTS with RF2.
> 
> The primary issues I'm experiencing are:
> 
> Light load against nodes in dc1 was causing OutOfMemory exceptions across all Cassandra servers outside of dc1 which were all idle and eventually after several hours happened on one of the dc1 nodes.  This issue was produced using svn trunk r1153002 and an in house written Snitched which effectively combined PropertyFileSnitch with some components of Ec2Snitch.  While trying to resolve these issues I have moved to a r1156490 snapshot and have switched across to just the PropertyFileSnitch and simply utilising the broadcast_address configuration option available in trunk which seems to work quite well.
> 
> Since moving to r1156490 we have stopped getting OOM's, but that may actually be because we have been unable to send traffic to the cluster to be able to produce one.
> 
> The most current issues I have been experiencing are the following:
> 
> 1) thrift timeouts & general degraded response times
> 2) *lots* of exception errors, such as:
> 
> ERROR [ReadRepairStage:1076] 2011-08-11 13:33:41,266 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadRepairStage:1076,5,main]
> java.lang.AssertionError
>        at org.apache.cassandra.service.RowRepairResolver.resolve(RowRepairResolver.java:73)
>        at org.apache.cassandra.service.AsyncRepairCallback$1.runMayThrow(AsyncRepairCallback.java:54)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> 
> 3) ring imbalances during a repair (refer to the above nodetool ring output)
> 4) regular failure detection when any node does something only moderately stressful, such as a repair or are under light load etc. but the node itself thinks it is fine.
> 
> My hosts are all 32Gb with either 4 or 16 cores, I've set heaps appropriately to half physical memory (16G) and for the purpose of cluster simplicity set all younggen to 400Mb.  JNA is in use, commitlogs and data have been split onto different filesystems and so on.
> 
> My data set as described by a dev is essentially as follows:
> 
> 3 column families (tables):
> 
> cf1.  The RowKey is the user id.  This is the primary column family queried on and always just looked up by RowKey.  It has 1 supercolumn called "seg".  The column names in this supercolumn are the segment_id's that the user belongs to and the value is just "1".  This should have about 150mm rows.  Each row will have an average of 2-3 columns in the "seg" supercolumn.  The column values have TTL's set on them.
> 
> cf2.  This is a CounterColumnFamily.  There's only a single "cnt" column which stores a counter of the number of cf1's having that segment.  This was only updated during the import and is not read at all.
> 
> cf3.  This is a lookup between the RowKey which is an external ID and the RowKey to be used to find the user in the cf1  CF.
> 
> 
> Does anyone have any ideas or suggestions about where I should be focusing on to get to the bottom of these issues or any recommendations on where I should be focusing my efforts on?
> 
> Thanks,
> Anton
> 
> 
> 
>