Re: Using a separate commit log drive was 4x slower

2010-08-10 Thread Peter Schuller
I have no explanation for the slower reads, but I have an hypothesis
on the writes.

Your iostat shows:

 Device: rrqm/s   wrqm/s r/s w/s   rsec/s   wsec/s avgrq-sz 
 avgqu-sz   await  svctm  %util
 cciss/c0d0    0.00   908.50    0.00  110.50 0.00  8152.00    
 73.77 0.57    5.20   4.93  54.50
 cciss/c0d1    0.00 0.00   16.50    0.00  1424.00 0.00    
 86.30 0.10    6.06   2.73   4.50
 dm-0  0.00 0.00    0.00 1019.00 0.00  8152.00 
 8.00 6.25    6.13   0.53  54.50
 dm-1  0.00 0.00    0.00    0.00 0.00 0.00 
 0.00 0.00    0.00   0.00   0.00

So that's keeping the disk busy around 50-60% of the time. This seems
roughly consistent with your commit batch window being set to 5 ms and
the system drive NOT being supported by a battery-backed cache (such
that an fsync() actually does have to wait for the data to be on the
platter).

Is your non-system drive backed by the BBU? (I'm not sure if the
controllers would support having some volumes backed by BBU protected
cache and not others.)

If it is the case that the other volume is BBU backed, then maybe the
slowdown on writes is due to this. In any case, whatever the situation
was before, the above stats do seem roughly consistent with
write-through fsync() and batch window of 5ms, given sufficient
concurrency to achieve the throughput you're seeing. On the other
hand, roughly consistent is not very precise, and the original
performance on the RAID:ed device is probably also roughly consistent
with this ;)

--
/ Peter Schuller


Re: non blocking Cassandra with Tornado

2010-08-10 Thread Ryan Daum


 Barring this we (place where I work, Chango) will probably eventually fork
 Cassandra to have a RESTful interface and use the Jetty async HTTP client to
 connect to it. It's just ridiculous for us to have threads and associated
 resources tied up on I/O-blocked operations.


 We've done exactly this but with Netty rather than Jetty. Helps too because
 we can easily have testers look at what we put into our CFs. Took some
 cheating to marshall the raw binaries into JSON but I'm pretty happy with
 what it's bought us so far.


Are you capable/willing/interested in sharing the work you did?

Ryan


Re: Growing commit log directory.

2010-08-10 Thread Jonathan Ellis
having 3 digit pending counts in both RRS and RMS is a danger sign.
It looks like you are i/o bound on reads, and possibly on writes as
well. (commitlog not on separate disk?)

On Mon, Aug 9, 2010 at 10:53 PM, Edward Capriolo edlinuxg...@gmail.com wrote:
 On Mon, Aug 9, 2010 at 8:20 PM, Jonathan Ellis jbel...@gmail.com wrote:
 what does tpstats or other JMX monitoring of the o.a.c.concurrent stages 
 show?

 On Mon, Aug 9, 2010 at 4:50 PM, Edward Capriolo edlinuxg...@gmail.com 
 wrote:
 I have a 16 node 6.3 cluster and two nodes from my cluster are giving
 me major headaches.

 10.71.71.56   Up         58.19 GB
 10827166220211678382926910108067277    |   ^
 10.71.71.61   Down       67.77 GB
 123739042516704895804863493611552076888    v   |
 10.71.71.66   Up         43.51 GB
 127605887595351923798765477786913079296    |   ^
 10.71.71.59   Down       90.22 GB
 139206422831293007780471430312996086499    v   |
 10.71.71.65   Up         22.97 GB
 148873535527910577765226390751398592512    |   ^

 The symptoms I am seeing are nodes 61 and nodes 59 have huge 6 GB +
 commit log directories. They keep growing, along with memory usage,
 eventually the logs start showing GCInspection errors and then the
 nodes will go OOM

 INFO 14:20:01,296 Creating new commitlog segment
 /var/lib/cassandra/commitlog/CommitLog-1281378001296.log
  INFO 14:20:02,199 GC for ParNew: 327 ms, 57545496 reclaimed leaving
 7955651792 used; max is 9773776896
  INFO 14:20:03,201 GC for ParNew: 443 ms, 45124504 reclaimed leaving
 8137412920 used; max is 9773776896
  INFO 14:20:04,314 GC for ParNew: 438 ms, 54158832 reclaimed leaving
 8310139720 used; max is 9773776896
  INFO 14:20:05,547 GC for ParNew: 409 ms, 56888760 reclaimed leaving
 8480136592 used; max is 9773776896
  INFO 14:20:06,900 GC for ParNew: 441 ms, 58149704 reclaimed leaving
 8648872520 used; max is 9773776896
  INFO 14:20:08,904 GC for ParNew: 462 ms, 59185992 reclaimed leaving
 8816581312 used; max is 9773776896
  INFO 14:20:09,973 GC for ParNew: 460 ms, 57403840 reclaimed leaving
 8986063136 used; max is 9773776896
  INFO 14:20:11,976 GC for ParNew: 447 ms, 59814376 reclaimed leaving
 9153134392 used; max is 9773776896
  INFO 14:20:13,150 GC for ParNew: 441 ms, 61879728 reclaimed leaving
 9318140296 used; max is 9773776896
 java.lang.OutOfMemoryError: Java heap space
 Dumping heap to java_pid10913.hprof ...
  INFO 14:22:30,620 InetAddress /10.71.71.66 is now dead.
  INFO 14:22:30,621 InetAddress /10.71.71.65 is now dead.
  INFO 14:22:30,621 GC for ConcurrentMarkSweep: 44862 ms, 261200
 reclaimed leaving 9334753480 used; max is 9773776896
  INFO 14:22:30,621 InetAddress /10.71.71.64 is now dead.

 Heap dump file created [12730501093 bytes in 253.445 secs]
 ERROR 14:28:08,945 Uncaught exception in thread Thread[Thread-2288,5,main]
 java.lang.OutOfMemoryError: Java heap space
        at 
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)
 ERROR 14:28:08,948 Uncaught exception in thread Thread[Thread-2281,5,main]
 java.lang.OutOfMemoryError: Java heap space
        at 
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:71)
  INFO 14:28:09,017 GC for ConcurrentMarkSweep: 33737 ms, 85880
 reclaimed leaving 9335215296 used; max is 9773776896

 Does anyone have any ideas what is going on?




 --
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com


 Hey guys thanks for the help. I had lowered my Xmx from 12GB to 10xmx
 because I saw:

 [r...@cdbsd09 ~]# /usr/local/cassandra/bin/nodetool --host 10.71.71.59
 --port 8585 info
 123739042516704895804863493611552076888
 Load             : 68.91 GB
 Generation No    : 1281407425
 Uptime (seconds) : 1459
 Heap Memory (MB) : 6476.70 / 12261.00

 This was happening:
 [r...@cdbsd11 ~]# /usr/local/cassandra/bin/nodetool --host
 cdbsd09.hadoop.pvt --port 8585 tpstats
 Pool Name                    Active   Pending      Completed
 STREAM-STAGE                      0         0              0
 RESPONSE-STAGE                    0         0          16478
 ROW-READ-STAGE                   64      4014          18190
 LB-OPERATIONS                     0         0              0
 MESSAGE-DESERIALIZER-POOL         0         0          60290
 GMFD                              0         0            385
 LB-TARGET                         0         0              0
 CONSISTENCY-MANAGER               0         0           7526
 ROW-MUTATION-STAGE               64       908         182612
 MESSAGE-STREAMING-POOL            0         0              0
 LOAD-BALANCER-STAGE               0         0              0
 FLUSH-SORTER-POOL                 0         0              0
 MEMTABLE-POST-FLUSHER             0         0              8
 FLUSH-WRITER-POOL                 0         0              8
 AE-SERVICE-STAGE                  0         0              0
 HINTED-HANDOFF-POOL               1         9            

Re: COMMIT-LOG_WRITER Assertion Error

2010-08-10 Thread Jonathan Ellis
Can you create a ticket for this?

On Mon, Aug 9, 2010 at 8:42 PM, Arya Goudarzi agouda...@gaiaonline.com wrote:
 I've never run 0.6. I have been running of trunc with automatic svn update 
 and build everyday at 2pm. One of my nodes got this error which lead to the 
 same last error prior to build and restart today. Hope this helps better:

 java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
 java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:549)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:174)
        at 
 org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:120)
        at 
 org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:90)
        at 
 org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:224)
 Caused by: java.util.concurrent.ExecutionException: 
 java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:252)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at 
 org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:545)
        ... 5 more
 Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
        at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:636)
 Caused by: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:408)
        at 
 org.apache.cassandra.db.ColumnFamilyStore$2.runMayThrow(ColumnFamilyStore.java:445)
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 6 more
 Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:252)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:400)
        ... 8 more
 Caused by: java.lang.AssertionError
        at 
 org.apache.cassandra.db.commitlog.CommitLogHeader$CommitLogHeaderSerializer.serialize(CommitLogHeader.java:157)
        at 
 org.apache.cassandra.db.commitlog.CommitLogHeader.writeCommitLogHeader(CommitLogHeader.java:124)
        at 
 org.apache.cassandra.db.commitlog.CommitLogSegment.writeHeader(CommitLogSegment.java:70)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegmentsInternal(CommitLog.java:450)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:75)
        at 
 org.apache.cassandra.db.commitlog.CommitLog$6.call(CommitLog.java:394)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
 org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:52)
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 1 more

 - Original Message -
 From: Jonathan Ellis jbel...@gmail.com
 To: user@cassandra.apache.org
 Sent: Monday, August 9, 2010 5:18:35 PM
 Subject: Re: COMMIT-LOG_WRITER Assertion Error

 Sounds like you upgraded to trunk from 0.6 without draining your
 commitlog first?

 On Mon, Aug 9, 2010 at 3:30 PM, Arya Goudarzi agouda...@gaiaonline.com 
 wrote:
 Just throwing this out there as it could be a concern. I had a cluster of 3 
 nodes running. Over the weekend I updated to trunc (Aug 9th @ 2pm). Today, I 
 came to run my daily tests and my client kept giving me TSocket timeouts. 
 Checking the error log of Cassandra servers, all 3 nodes had this and they 
 all became unresponsive! Not sure how to reproduce this but a restart of all 
 3 nodes fixed the issue:

 ERROR [COMMIT-LOG-WRITER] 2010-08-09 11:30:27,722 CassandraDaemon.java (line 
 82) Uncaught exception in thread Thread[COMMIT-LOG-WRITER,5,main]
 java.lang.AssertionError
        at 
 

why does it take 60-90 seconds for a new node to get up?

2010-08-10 Thread S Ahmed
Why is it that, if you set AutoBootStrap = false that it takes 60-90 seconds
for the node to announce itself?

I just want to understand what is going on during that time, and why that
specific timeframe (if there is a reason?)


Re: explanation of generated files and ops

2010-08-10 Thread Peter Schuller
 Is this pretty much all the files that Cassandra generates? (have I missed
 any)

I believe so. There will also be some temporary files (*-tmp*) during
compaction, and you'll see some *.Compacted marker files (empty IIRC).

 What exactly is stored in the -Filter.db files?

Those are bloom filters:

http://wiki.apache.org/cassandra/ArchitectureOverview
http://spyced.blogspot.com/2009/01/all-you-ever-wanted-to-know-about.html

-- 
/ Peter Schuller


Re: COMMIT-LOG_WRITER Assertion Error

2010-08-10 Thread Arya Goudarzi
Sure. https://issues.apache.org/jira/browse/CASSANDRA-1376

- Original Message -
From: Jonathan Ellis jbel...@gmail.com
To: user@cassandra.apache.org
Sent: Tuesday, August 10, 2010 7:05:31 AM
Subject: Re: COMMIT-LOG_WRITER Assertion Error

Can you create a ticket for this?

On Mon, Aug 9, 2010 at 8:42 PM, Arya Goudarzi agouda...@gaiaonline.com wrote:
 I've never run 0.6. I have been running of trunc with automatic svn update 
 and build everyday at 2pm. One of my nodes got this error which lead to the 
 same last error prior to build and restart today. Hope this helps better:

 java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
 java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:549)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:174)
        at 
 org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:120)
        at 
 org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:90)
        at 
 org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:224)
 Caused by: java.util.concurrent.ExecutionException: 
 java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:252)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at 
 org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:545)
        ... 5 more
 Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
        at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:636)
 Caused by: java.lang.RuntimeException: 
 java.util.concurrent.ExecutionException: java.lang.AssertionError
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:408)
        at 
 org.apache.cassandra.db.ColumnFamilyStore$2.runMayThrow(ColumnFamilyStore.java:445)
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 6 more
 Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:252)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:400)
        ... 8 more
 Caused by: java.lang.AssertionError
        at 
 org.apache.cassandra.db.commitlog.CommitLogHeader$CommitLogHeaderSerializer.serialize(CommitLogHeader.java:157)
        at 
 org.apache.cassandra.db.commitlog.CommitLogHeader.writeCommitLogHeader(CommitLogHeader.java:124)
        at 
 org.apache.cassandra.db.commitlog.CommitLogSegment.writeHeader(CommitLogSegment.java:70)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegmentsInternal(CommitLog.java:450)
        at 
 org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:75)
        at 
 org.apache.cassandra.db.commitlog.CommitLog$6.call(CommitLog.java:394)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
 org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:52)
        at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 1 more

 - Original Message -
 From: Jonathan Ellis jbel...@gmail.com
 To: user@cassandra.apache.org
 Sent: Monday, August 9, 2010 5:18:35 PM
 Subject: Re: COMMIT-LOG_WRITER Assertion Error

 Sounds like you upgraded to trunk from 0.6 without draining your
 commitlog first?

 On Mon, Aug 9, 2010 at 3:30 PM, Arya Goudarzi agouda...@gaiaonline.com 
 wrote:
 Just throwing this out there as it could be a concern. I had a cluster of 3 
 nodes running. Over the weekend I updated to trunc (Aug 9th @ 2pm). Today, I 
 came to run my daily tests and my client kept giving me TSocket timeouts. 
 Checking the error log of Cassandra servers, all 3 nodes had this and they 
 all became unresponsive! Not sure how to reproduce this but a restart of all 
 3 

Re: Using a separate commit log drive was 4x slower

2010-08-10 Thread Jeremy Davis
Yeah, it has a BBU, and it is charged and on..
Very odd behavior, I'm stumped.

-JD

On Tue, Aug 10, 2010 at 12:28 AM, Peter Schuller 
peter.schul...@infidyne.com wrote:

 I have no explanation for the slower reads, but I have an hypothesis
 on the writes.

 Your iostat shows:

  Device: rrqm/s   wrqm/s r/s w/s   rsec/s   wsec/s
 avgrq-sz avgqu-sz   await  svctm  %util
  cciss/c0d00.00   908.500.00  110.50 0.00  8152.00
 73.77 0.575.20   4.93  54.50
  cciss/c0d10.00 0.00   16.500.00  1424.00 0.00
 86.30 0.106.06   2.73   4.50
  dm-0  0.00 0.000.00 1019.00 0.00  8152.00
 8.00 6.256.13   0.53  54.50
  dm-1  0.00 0.000.000.00 0.00 0.00
 0.00 0.000.00   0.00   0.00

 So that's keeping the disk busy around 50-60% of the time. This seems
 roughly consistent with your commit batch window being set to 5 ms and
 the system drive NOT being supported by a battery-backed cache (such
 that an fsync() actually does have to wait for the data to be on the
 platter).

 Is your non-system drive backed by the BBU? (I'm not sure if the
 controllers would support having some volumes backed by BBU protected
 cache and not others.)

 If it is the case that the other volume is BBU backed, then maybe the
 slowdown on writes is due to this. In any case, whatever the situation
 was before, the above stats do seem roughly consistent with
 write-through fsync() and batch window of 5ms, given sufficient
 concurrency to achieve the throughput you're seeing. On the other
 hand, roughly consistent is not very precise, and the original
 performance on the RAID:ed device is probably also roughly consistent
 with this ;)

 --
 / Peter Schuller



Re: Using a separate commit log drive was 4x slower

2010-08-10 Thread Jonathan Ellis
Other activity, e.g. syslog?

Journaling at the FS level? you could try making a small partition
formatted as ext2.

On Tue, Aug 10, 2010 at 5:10 PM, Jeremy Davis
jerdavis.cassan...@gmail.com wrote:
 Yeah, it has a BBU, and it is charged and on..
 Very odd behavior, I'm stumped.

 -JD

 On Tue, Aug 10, 2010 at 12:28 AM, Peter Schuller
 peter.schul...@infidyne.com wrote:

 I have no explanation for the slower reads, but I have an hypothesis
 on the writes.

 Your iostat shows:

  Device: rrqm/s   wrqm/s r/s w/s   rsec/s   wsec/s
  avgrq-sz avgqu-sz   await  svctm  %util
  cciss/c0d0    0.00   908.50    0.00  110.50 0.00  8152.00
  73.77 0.57    5.20   4.93  54.50
  cciss/c0d1    0.00 0.00   16.50    0.00  1424.00 0.00
  86.30 0.10    6.06   2.73   4.50
  dm-0  0.00 0.00    0.00 1019.00 0.00  8152.00
  8.00 6.25    6.13   0.53  54.50
  dm-1  0.00 0.00    0.00    0.00 0.00 0.00
  0.00 0.00    0.00   0.00   0.00

 So that's keeping the disk busy around 50-60% of the time. This seems
 roughly consistent with your commit batch window being set to 5 ms and
 the system drive NOT being supported by a battery-backed cache (such
 that an fsync() actually does have to wait for the data to be on the
 platter).

 Is your non-system drive backed by the BBU? (I'm not sure if the
 controllers would support having some volumes backed by BBU protected
 cache and not others.)

 If it is the case that the other volume is BBU backed, then maybe the
 slowdown on writes is due to this. In any case, whatever the situation
 was before, the above stats do seem roughly consistent with
 write-through fsync() and batch window of 5ms, given sufficient
 concurrency to achieve the throughput you're seeing. On the other
 hand, roughly consistent is not very precise, and the original
 performance on the RAID:ed device is probably also roughly consistent
 with this ;)

 --
 / Peter Schuller





-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com


Re: Using a separate commit log drive was 4x slower

2010-08-10 Thread Peter Schuller
 Yeah, it has a BBU, and it is charged and on..
 Very odd behavior, I'm stumped.

I advise double-checking raid volume settings and ensuring that policy
is truly such that the write cache is used. This may also be a
function of kernel driver settings depending on what RAID
controller/kernel version you have (for example make sure that an
fsync() doesn't result in asking the RAID controller to flush caches
regardless of BBU state). In any case, these stats:

Device: rrqm/s   wrqm/s r/s w/s   rsec/s   wsec/s
avgrq-sz avgqu-sz   await  svctm  %util
cciss/c0d00.00   908.500.00  110.50 0.00  8152.00
73.77 0.575.20   4.93  54.50

... are highly inconsistent with write-back caching under the
assumption that the writes are indeed the sequential writes to the
commit log. Write counts in the ~ 100/second ballpark, with no reads,
average request size of 74, an average transaction time of 4.93 and a
utilization of 54% *really really* sounds like the I/O is going all
the way down to the platter. Either that or the RAID firmware/driver
is not doing its job properly.

I've attached a small script (note: UGLY hack since I just whipped it
up, not a proper tool, but it does the job for this) that you can run
to test it:

   ./writelat.py /path/to/file_to_write_to # warning, file will be
truncated/destroyed if it exists

If you run this on an idle system and you're truly doing write-back
caching, you should see numbers BELOW 1 (i.e., sub-millisecond times)
(but you can ignore the first sample probably).

It writes one 8kb chunk of data, fsync:s, writes another 8kb, fsyncs,
etc. I predict you'll see numbers in the 3-7 millisecond range.

Sample output for me with a plain old SATA drive (and ZFS on FreeBSD) follows.

33.8141918182
8.32605361938
8.44812393188
8.44788551331
8.40210914612
8.4490776062
8.38303565979
8.57901573181
8.20922851562
8.21614265442
10.0581645966
8.37683677673
8.50605964661
8.376121521
9.86790657043
8.43715667725
8.45789909363
8.40520858765
8.4171295166
8.46195220947
8.41498374939
8.46099853516
8.44287872314
8.43000411987
8.455991745

-- 
/ Peter Schuller
#!/usr/bin/env python

# ugly hack, reader beware

import os
import sys
import time

COUNT = 25

BLOCK = ''.join('x' for _ in xrange(8*1024))

f = file(sys.argv[1], 'w')
for _ in xrange(COUNT):
start_time = time.time()
f.write(BLOCK)
f.flush()
os.fsync(f.fileno())
stop_time = time.time()
# uncomment to test-rewriting the same block repeatedly
# f.seek(0)
print (stop_time - start_time) * 1000
f.close()


Re: explanation of generated files and ops

2010-08-10 Thread Robert Coli

On 8/9/10 9:00 PM, S Ahmed wrote:
Is this pretty much all the files that Cassandra generates? (have I 
missed any)
If you are running cassandra via the linux init scripts, you are setting 
outfile for jsvc to be :


-outfile /var/log/$NAME/output.log \


And with :

-errfile 1 \


You get both what is on STDOUT and STDERR in your outfile. If you have 
Java GC debugging enabled, for example, it will be logged here.


=Rob



Soliciting thoughts on possible read optimization

2010-08-10 Thread Arya Asemanfar
I mentioned this today to a couple folks at Cassandra Summit, and thought
I'd solicit some more thoughts here.

Currently, the read stage includes checking row cache. So if your concurrent
reads is N and you have N reads reading from disk, the next read will block
until a disk read finishes, even if it's in row cache. Would it make sense
to isolate disk reads from cache reads? To either make the read stage be
only used on misses, or to make 2 read stages CacheRead and DiskRead? Of
course, we'd have to go to DiskRead for mmap since we wouldn't know until we
asked the OS.

My thought is that stages should be based on resources rather than
semantics, but that may be wrong. Logically, I don't think it would make
sense to have the read stage bounded in a hypothetical system where there is
no IO; it's most likely because of the disk and subsequent IO contention
that that cap was introduced.

As a possible bonus with this change, you can make other optimizations like
batching row reads from disk where the keys were in key cache (does this
even make sense? I'm not too sure how that would work).

Let me know what you guys think.

Thanks,
Arya


Re: explanation of generated files and ops

2010-08-10 Thread Robert Coli

On 8/9/10 9:00 PM, S Ahmed wrote:

What exactly is stored in the -Filter.db files?

Oh, didn't see this line.

http://wiki.apache.org/cassandra/ArchitectureOverview

Bloom filter (all keys in data file). A Bloom filter, is a 
space-efficient probabilistic data structure that is used to test 
whether an element is a member of a set. False positives are possible, 
but false negatives are not. Cassandra uses bloom filters to save IO 
when performing a key lookup: each SSTable has a bloom filter associated 
with it that Cassandra checks before doing any disk seeks, making 
queries for keys that don't exist almost free.



=Rob