i did check sstables, and there are only three. i haven't done any
major compacts.
do u think it is taking so long because it must sift thru the deleted
columns before compaction?
so accessing a column by name instead of slice predicate is faster?
On 08/24/2010 11:23 PM, Benjamin Black wrote:
Todd,
This is a really bad idea. What you are likely doing is spreading
that single row across a large number of sstables. The more columns
you insert, the more sstables you are likely inspecting, the longer
the get_slice operations will take. You can test whether this is so
by running nodetool compact when things start slowing down. If it
speeds up, that is likely the problem. If you are deleting that much,
you should also tune GCGraceSeconds way down (from the default of 10
days) so the space is reclaimed on major compaction and, again, there
are fewer things to inspect.
Long rows written over long periods of time are almost certain to give
worse read performance, even far worse, than rows written all at once.
b
On Tue, Aug 24, 2010 at 10:17 PM, B. Todd Burruss<bburr...@real.com> wrote:
thx artie,
i haven't used a super CF because i thought it has more trouble doing slices
because the entire row must be deserialized to get to the subcolumn you
want?
iostat is nothing, 0.0. i have plenty of RAM and the OS is I/O caching
nicely
i haven't used the key cache, because i only have one key, the row of the
queue ;)
i haven't used row cache because i need the row to grow quite large,
millions of columns. and the size of data could be arbitrary - right now i
am testing with< 32 byte values per column.
i do need quorum consistency.
i have read previous that some folks are using a single row with millions of
columns. is anyone using get_slice to pick off the first or the last column
in the row?
On 08/24/2010 09:25 PM, Artie Copeland wrote:
Have you tried using a super column, it seems that having a row with over
100K columns and growing would be alot for cassandra to deserialize? what
is iostat and jmeter telling you? it would be interesting to see that data.
also what are you using for you key or row caching? do you need to use a
quorum consistency as that can slow down reads as well, can you use a lower
consistency level?
Artie
On Tue, Aug 24, 2010 at 9:14 PM, B. Todd Burruss<bburr...@real.com> wrote:
i am using get_slice to pull columns from a row to emulate a queue.
column names are TimeUUID and the values are small,< 32 bytes. simple
ColumnFamily.
i am using SlicePredicate like this to pull the first ("oldest") column in
the row:
SlicePredicate predicate = new SlicePredicate();
predicate.setSlice_range(new SliceRange(new byte[] {}, new byte[]
{}, false, 1));
get_slice(rowKey, colParent, predicate, QUORUM);
once i get the column i remove it. so there are a lot of gets and
mutates, leaving lots of deleted columns.
get_slice starts off performing just fine, but then falls off dramatically
as the number of columns grows. at its peak there are 100,000 columns and
get_slice is taking over 100ms to return.
i am running a single instance of cassandra 0.7 on localhost, default
config. i've done some googling and can't find any tweaks or tuning
suggestions specific to get_slice. i already know about separating
commitlog and data, watching iostat, GC, etc.
any low hanging tuning fruit anyone can think of? in 0.6 i recall an
index for columns, maybe that is what i need?
thx
--
http://yeslinux.org
http://yestech.org