I ran another test last night with the build dated 29 Nov 2009. Other than the
Cassandra version, the setup was the same as before. I got qualitatively
similar results as before, too -- the read latency increased fairly smoothly
from 250ms to 1s, the GC times reported by jconsole are low, the
i am seeing this as well. i did a test with just 1 cassandra node,
ReplicationFactor=1, 'get' ConsistencyLevel.ONE, 'put'
ConsistencyLevel.QUORUM. The first test was writing and reading random
values starting from a fresh database. The put performance is staying
reasonabe, but the read
i would expect read latency to increase linearly w/ the number of
sstables you have around. how many are in your data directories? is
your compaction lagging 1000s of tables behind again?
On Thu, Dec 3, 2009 at 12:58 PM, Freeman, Tim tim.free...@hp.com wrote:
I ran another test last night with
Tim,
Very interesting information. Was there any other numbers in tpstats
from nodeprobe that are growing?
Can you plot the number of SSTables? Are you using the standard
storage-conf.xml defaults?
We've seen reads spike like this with a large number of SSTables.
-Chris
On Dec 3, 2009,
how many are in your data directories? is your compaction
lagging 1000s of tables behind again?
Yes, there are 2348 files in data/Keyspace1, and jconsole says the compaction
pool has 1600 pending tasks.
Chris Goffinet's questions were good too but it will take me a little while to
get
On Thu, Dec 3, 2009 at 1:04 PM, Chris Goffinet goffi...@digg.com wrote:
We've seen reads spike like this with a large number of SSTables.
(Which I emphasize is a natural consequence of the SSTable design.)
On Thu, Dec 3, 2009 at 1:11 PM, Freeman, Tim tim.free...@hp.com wrote:
how many are in your data directories? is your compaction
lagging 1000s of tables behind again?
Yes, there are 2348 files in data/Keyspace1, and jconsole says the compaction
pool has 1600 pending tasks.
If you stop doing
Tim,
After you stop the test. Do you see the pending tasks for compaction
drop? Need to verify you didn't run into a new bug. If the number
starts to slowly drop that just indicates that compactions are not
keeping up with your write levels.
On Dec 3, 2009, at 11:11 AM, Freeman, Tim
i do not have any pending tasks in the compaction pool but i have 1164
files in my data directory. one thing to note about my situation is
that i did run out of disk space during my test. cassandra _seemed_ to
recover nicely.
tim, is your's recovering? i plan to rerun the test tonight with a
On Thu, Dec 3, 2009 at 1:34 PM, Jonathan Ellis jbel...@gmail.com wrote:
On Thu, Dec 3, 2009 at 1:32 PM, B. Todd Burruss bburr...@real.com wrote:
i do not have any pending tasks in the compaction pool but i have 1164
files in my data directory.
how many CFs are those spread across?
... and
files with Data in their name = 384
files with Compacted in their name = 11
one CF, my keyspace is like this:
Keyspaces
Keyspace Name=uds
KeysCachedFraction0.01/KeysCachedFraction
ColumnFamily CompareWith=BytesType Name=bucket /
/Keyspace
/Keyspaces
i've also used
I also like this feature, I also use keyspaces as another dimension
for some applications. I also don't worry about securing cassandra
as I run behind firewalls. I can see the argument for authentication
per keyspace, but I like it as optional.
For instance consider the use case where you have
nothing - i have DEBUG level set and bounced the server. i'll restart
again.
cassandra spends a lot of time loading INDEXes
On Thu, 2009-12-03 at 14:08 -0600, Jonathan Ellis wrote:
what does it log when you nodeprobe compact, with debug logging on?
On Thu, Dec 3, 2009 at 1:59 PM, B. Todd
I am evaluating Cassandra and playing with some data model ideas. One
model I'm thinking about is to take privacy concerns into account, such
that interleaving tenant data in the same column family is not allowed.
Another reason I can think to keep the data for different tenants in
separate CFs
On Thu, Dec 3, 2009 at 3:22 PM, Coe, Robin robin@bluecoat.com wrote:
I seem to recall some talk about dynamically setting a key space or maybe it
was dynamically loading column families? Is this already a feature request,
to your knowledge, or should I open a Jira issue?
Yup, this looks like pretty good coverage of the topic.
I *think* I like the idea of Cassandra pushing a CF change to its peers, as
opposed to managing it by a separate admin task, simply because I wouldn't want
a change managed by an application admin to be missed because of bad
On Thu, Dec 3, 2009 at 4:23 PM, Coe, Robin robin@bluecoat.com wrote:
I *think* I like the idea of Cassandra pushing a CF change to its peers, as
opposed to managing it by a separate admin task, simply because I wouldn't
want a change managed by an application admin to be missed because of
Hi all, I was reading the original paper[1] looking for answers to our
problem of somehow handling transaction in Cassandra, and I read that future
works involves the ability to support atomicity across keys and secondary
index.
does anyone has any update on that? Whatever information regarding
Looking at jconsole I see a high number of writes when I do removes,
so I am guessing these are tombstones being written? If that's the
case, is the data being removed and replaced by tombstones? and will
they all be deleted eventually when compaction runs?
On Thu, Dec 3, 2009 at 3:18 PM, Ramzi
cassandra never modifies data in-place. so it writes tombstones to
supress the older writes, and when compaction occurs the data and
tombstones get GC'd (after the period specified in your config file).
On Thu, Dec 3, 2009 at 8:07 PM, Ramzi Rabah rra...@playdom.com wrote:
Looking at jconsole I
Dear all,
I know in get_slice method, we have parameter reversed which has default
value false.
What does cassandra do when I set reversed = true? How does this
difference with reversed = false?
Thank a lot for support.
--
Best regards,
JKnight
New error in cassandra - cachetable-timer anyone encounter this thanks
at
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:177)
at
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:252)
at
Thanks but that is not.
I want to know how Cassandra server handle, not client example.
Which reversed value is better for server performance?
Thanks.
On Thu, Dec 3, 2009 at 10:59 PM, Jonathan Ellis jbel...@gmail.com wrote:
Is this what you're looking for?
false is slightly more performant
look at http://wiki.apache.org/cassandra/ArchitectureInternals and
start reading the classes in Read Path if you want to see the code
On Thu, Dec 3, 2009 at 11:06 PM, JKnight JKnight beukni...@gmail.com wrote:
Thanks but that is not.
I want to know how
24 matches
Mail list logo