Its technically possible to have without breaking basic levelDB
algorithm configurable sstable size and count on different levels?
something like:
level 1 - 10 x 50 MB tables
level 2 - 60 x 40 MB tables
level 3 - 150 x 30 MB tables
I am interested in more deeper leveldb research, because curre
> Looking at heap dumps, a lot of memory is taken by memtables, much
more than 1/3 of heap. At the same time, logs say that it has nothing to
flush since there are not dirty memtables.
I seen this too.
> So, what are cassandra memory requirement? Is it 1% or 2% of disk data?
It depends on numbe
Then for each read, Cassandra will go through all the SSTables (or
one SSTable in each level for the leveled compaction strategy)? How to deal
with this
problem?
bloom filters can guess right sstables to be read with high probability
< 0.1%. In reality even if you are using size based compacti
currently j.o.a.c.io.sstable.indexsummary is implemented as ArrayList of
KeyPosition (RowPosition key, long offset)
i propose to change it to
RowPosition keys[]
long offsets[]
this will lower number of java objects used per entry from 2
(KeyPosition + RowPosition) to 1.
For building these ar
> How large is the bloom filters in total? I.e., sizes of the
*-Filter.db files.
On moderate node about 6.5 GB, index sampling will be about 4 GB, heap
12 gb.
> In general, don't expect to be able to run at close to heap capacity;
there *will* be spikes.
i try to tune for 80% of heap.
demo, it will be in cassandra 1.0.7
standard cassa bloom filter
-rw-r--r-- 1 root wheel 19307376721 Dec 27 20:06 sipdb-hc-4634-Data.db
-rw-r--r-- 1 root wheel 63 Dec 27 20:06
sipdb-hc-4634-Digest.sha1
-rw-r--r-- 1 root wheel770714896 Dec 27 20:06 sipdb-hc-4634-Filter.db
-rw
But just to be extra clear: Data will not actually be removed once the
row in question participates in compaction. Compactions will not be
actively triggered by Cassandra for tombstone processing reasons.
leveled compaction is really good for this because it compacts often
> But is there any way of implementing minimum required ACID subset on
top of Cassandra?
try this, its nosql ACID compliant. I haven't tested this, it will have
most likely pretty slow writes and lot of bugs like any other oracle
application.
http://www.oracle.com/technetwork/database/nosqld
> That is a good reason for both to be configurable IMO.
index sampling is currently configurable only per node, it would be
better to have it per Keyspace because we are using OLTP like and OLAP
keyspaces in same cluster. OLAP Keyspaces has about 1000x more rows.
But its difficult to estimate
makes me feel disappointed about consistency in Cassandra, but I wonder is
there is a way to work around it.
cassandra is not suitable for this kind of programs. CouchDB is slightly
better, it has transactions but no locking and i am not sure if
transaction isolation is supported now. mongodb
I don't know what you are basing that on. It seems unlikely to me that
the working set of a compaction is 600 MB. However, it may very well
be that the allocation rate is such that it contributes to an
additional 600 MB average heap usage after a CMS phase has completed.
I will investigate situa
I suggest you describe exactly what the problem is you have and why you
think stopping compaction/repair is the appropriate solution.
compacting 41.7 GB CF with about 200 millions rows adds - 600 MB to
heap, node logs messages like:
WARN [ScheduledTasks:1] 2011-12-27 00:20:57,972 GCInspector
If node is low on memory 0.95+ heap used it can do:
1. stop repair
2. stop largest compaction
3. reduce number of compaction slots
4. switch compaction to single threaded
flushing largest memtable/ cache reduce is not enough
my missunderstanding of FP ratio was based on assumption that ratio is
counted from node start, while it is getRecentBloomFilterFalseRatio()
> I don't understand how you reached that conclusion.
On my nodes most memory is consumed by bloom filters. Also 1.0 creates
larger bloom filters than 0.
Dne 25.12.2011 20:58, Peter Schuller napsal(a):
Read Count: 68844
[snip]
why reported bloom filter FP ratio is not counted like this
10/68844.0
0.00014525594096798558
Because the read count is total amount of reads to the CF, while the
bloom filter is per sstable. The number
I have following CF
Read Count: 68844
Read Latency: 9.942 ms.
Write Count: 209712
Write Latency: 0.297 ms.
Pending Tasks: 0
Bloom Filter False Postives: 10
Bloom Filter False Ratio: 0.0
cassandra read performance depends on your disk cache (free memory at
node not used by cassandra) and disk IOPS peformance. In ideal case (no
need to merge sstables) cassandra needs 2 IOPS per data read if
cassandra key/row caches are not used.
Standard hard drive has about 150 IOPS. If you ha
Dne 11.11.2011 7:55, Radim Kolar napsal(a):
i have problem with large CF (about 200 billions entries per node).
While i can configure index_interval to lower memory requirements, i
still have to stick with huge bloom filters.
Ideal would be to have bloom filters configurable like in hbase
Dne 14.12.2011 1:15, Maxim Potekhin napsal(a):
Thanks. It could be hidden from a human operator, I suppose :)
I agree. Open JIRA for it.
Possible, but unlikely. See
https://issues.apache.org/jira/browse/CASSANDRA-3537 for an example of
a "memory leak" that wasn't.
I didnt get the point. I have slowly increasing memory load on node, no
flushable memtables. How it could not be memory leak? Also running
nodetool upgradesstables
and rows forever stuck in HintsColumnFamily
You need to remove the hints data files to clear out the incomplete
hints from< 1.0.3;
I did. hints there are slowly increasing. I checked it today.
I have problem with reproducible memory leaks in 1.0.5 - node ends with
OOM, it worked fine with 1.0.3. Can someone write guide how to report
memory leaks - i can collect jmap dumps. It would be best to write wiki
page about it.
I am not happy with 1.x branch stability. It has memory leaks and
C:\cassandra\bin>nodetool -h 10.0.0.9 repair
Starting NodeTool
Error connection to remote JMX agent!
java.rmi.ConnectException: Connection refused to host: 192.168.140.1;
nested exc
eption is:
java.net.ConnectException: Connection timed out: connect
at
sun.rmi.transport.tcp.TCPE
Dne 1.12.2011 23:30, Bill napsal(a):
> Our largest dataset has 1200 billion rows.
Radim, out of curiosity, how many nodes is that running across?
32
I understand that my computer may be not as powerful as those used in
the other benchmarks,
but it shouldn't be that far off (1:30), right?
cassandra has very fast writes. you can have read:write ratios like 1:1000
pure read workload on 1 billion rows without key/row cache on 2 node cluster
R
1.0 branch is less stable then 0.8 for production. We discovered
following problems:
1. memory leak in scrub (also reported on this list)
2. problem with saving key caches for super column family - CASSANDRA-3511
3. in 1.0.3 some hints are stuck in system tables. Hints to other nodes
seems t
Dne 17.11.2011 17:42, Dan Hendry napsal(a):
What do you mean by ' better file offset caching'? Presumably you mean
'better page cache hit rate'?
fs metadata used to find blocks in smaller files are cached better.
Large files are using indirect blocks and you need more reads to find
correct bloc
Is there some simple way how to split large sstable into several smaller
ones? I increased min_compaction_threshold (smaller tables seems to get
better file offset caching from OS) and now i need to reshuffle data to
smaller sstables, running several cluster wide repairs worked well just
large
Dne 15.11.2011 22:04, Mick Semb Wever napsal(a):
But another node (on the same machine but different cluster), even after
an upgrade to the staging 1.0.3 and a `nodetool scrub`, always soaks all
available memory (up to and plateau at 30G). In fact no cf there use
compression anymore.
I had simila
It may be the case that your CL is the issue. You are writing it at
ONE, which means that out of the 4 replicas of that key (two in each
data center), you are only putting it on one of them.
cassandra will always try to replicate key to all available replicas.
Under normal conditions if you do
I am new to cassandra. I search for random write examples
you can access cassandra data at any node and keys can be accessed at
random.
from log output it seems that during hintedhandoff delivery compaction
is kicked too soon. There needs to be some delay for flusher to write
sstable.
INFO [GossipStage:1] 2011-11-14 13:16:03,933 Gossiper.java (line 745)
InetAddress /***.99.40 is now UP
INFO [HintedHandoff:1] 2011-11-14 1
Well to be honest I was thinking of using that connection in
production, not for a backup node.
For productions. there are several problems. Added network latency which
is inconsistent and vary greatly during day, sometimes you will face
network lags which will break cluster for a while (abo
I thought currently no one is maintaining supercolumns related code,
and also it not quite efficient.
yes. but they have nice features. you can do slices without need to use
orderedpartitioner
We store multi gigabyte sized files in cassandra chunked to 10 MB parts.
Storage is cheap today and replication will make your reads faster.
Dne 11.11.2011 19:14, M Vieira napsal(a):
Has anyone experimented running cassandra clusters in geographicly
separated locations connected thru ordinary broadband?
By ordinary broadband I mean 30Mbps or 50Mbps
for backup purposes, like place 1 replica on remote location over WAN??
yes. it works
It would be good to have index_interval configurable per keyspace.
Preferably in cassandra.yaml because i use it as tuning on nodes running
out of memory without affecting performance noticeably.
i have problem with large CF (about 200 billions entries per node).
While i can configure index_interval to lower memory requirements, i
still have to stick with huge bloom filters.
Ideal would be to have bloom filters configurable like in hbase.
Cassandra standard is about 1.05% false possiti
Dne 11.11.2011 5:58, Yan Chunlu napsal(a):
I think cassandra is doing great job on key-value data store, it saved
me tremendous work on maintain the data consistency and service
availability.But I think it would be great if it could support
more data structures such as key-list, currently I
Dne 10.11.2011 22:18, Dan Hendry napsal(a):
Is it possible to downgrade back to 0.8? Is there any way to convert 'h'
version SSTables to the old 'g' version? Any other data file changes to be
aware of?
try to add 0.8 node to cluster and decommission 1.0 node. maybe 0.8 will
understand streams fr
Dne 10.11.2011 8:16, Maki Watanabe napsal(a):
I missed the news.
How the "nodetool move" work in recent version (0.8.x or later?)
Just stream appropriate range of data between nodes?
yes
i consulted with hadoop expert and he told me that he is using value 100
for merging segments. I will rerun tests with 100 to check.
I found in stress tests that default setting this to 32 is way too high.
Hadoop guys are using value 10 during merge sorts to not stress IO that
much. I also discovered that filesystems like ZFS are using default io
queue size of 10 per drive.
I tried run tests with 10, 15 and 32 and there is
that's why disabling gossip + flush is better than drain. we should
probably remove it.
drain could be good if there is way to undrain node - to switch it back
into r/w.
Implement nodetool shutdown which will work like we are trying. First
stop gossip then wait for other nodes to see it dow
drain switches node into read only, which is undesirable because other
nodes still see it up.
mine issue has different stacktrace
If you are still hunting 1.0 assert during hintedhandoff, i can
reproduce it anytime. both nodes are 1.0.2. And just hint, we are using
super column families.
INFO [HintedHandoff:1] 2011-11-08 21:33:54,118
HintedHandOffManager.java (line 268) Started hinted handoff for token: 0
with IP: /***
Ooops, sorry about that. I overlooked the drain. Sorry for the misinformation!
cassandra still replays log file even on clean shutdown via nodetool
drain. It usually takes a while. I don't think it has concept of
clean-shutdown like SQL databases.
THEN, after that point, you have to wait for the 10 seconds to be
reasonably sure the commit log has been flushed.
nodetool drain flushes memtables before returning. its probably safe to
kill it after drain without waiting for commit log flush.
For things like rolling restarts, we do:
disablethrift
disablegossip
(...wait for all nodes to see this node go down..)
drain
i implemented this in our batch scripts for cassandra
disablegossip
sleep 10 seconds
dissablethrift
drain
KILL -TERM
similar thing should be added to bin/stop-server
Currently cassandra/conf/cassandra-env.sh disables use of jamm if
openjdk is detected.
I enabled it and tested it on openjdk 1.6 b23 and it works as expected.
That openjdk test can be probably removed.
> So my question related deterministic sharding is this, "what
rebalance feature(s) would be useful or needed once the partitions get
unbalanced?"
In current cassandra you can use "nodetool move" for rebalancing. Its
fast operation, portion of existing data is moved to new server.
Take a look at this:
http://www.oracle.com/technetwork/database/nosqldb/overview/index.html
> I understand the limitation/advantages of the architecture.
Read this http://en.wikipedia.org/wiki/CAP_theorem
Yeah, I can use HBase too.
but why you are not using hbase if its feature set fits your needs
better and want to have same functionality in cassandra? Its good that
both projects are different in this area. From rest of your post it
looks like you want to have cassandra ACID compliant, which i
- support for atomic operations or batches (if QUORUM fails, data should
not be visible with ONE)
zookeeper is solving that.
- TTL on CF, rows and counters
TTL on counters will be nice, but i am good with rest as it is
Anybody got success with running cassandra jna on freebsd 8? i got
following error
INFO [main] 2011-11-04 23:31:53,963 CLibrary.java (line 65) Unable to
link C library. Native methods will be disabled.
INFO [main] 2011-11-04 23:31:54,307 CLibrary.java (line 106) JNA
mlockall successful
Dne 4.11.2011 16:16, Rene Kochen napsal(a):
I'm using Cassandra 0.7.9.
Ok, so in this version, Cassandra waits with compaction. But when (in my
original example) are the four 1GB files compacted?
There are compacted when next file of similar size to 1 GB is created
What makes Cassandra decide to wait with this minor compaction?
What version do you using? There were some patch for 1.x branch which
will do it as you expect. Cassandra 0.8 waited with compactions.
* Compaction is expensive
Yes, it is. Thats why i deciced not to go with hadoop hdfs backed by
cassandra.
Dne 30.10.2011 23:34, Sorin Julean napsal(a):
Hey Chris,
Thanks for sharing all the info.
I have few questions:
1. What are you doing with so much memory :) ?
cassandra eats memory like there is no tomorrow on large databases. It
keeps some structures in memory which depends on database siz
bug is still there, i opened
https://issues.apache.org/jira/browse/CASSANDRA-3415 with command how to
reproduce.
Dne 18.10.2011 22:35, aaron morton napsal(a):
Looks like the column meta for the CF specifies a column name that is
not a valid Long. I seem to remember a bug like this something in the
past.
but why are columns in system tables untyped? if it was declared as
long, its impossible to insert bad
Look in jconcole -> org.apache.cassandra.db -> ColumnFamilies
bloom filter false ratio is on this server 0.0018 and 0,06% reads hits
more than 1 sstable.
From cassandra point of view, it looks good.
Dne 17.10.2011 22:06, aaron morton napsal(a):
Hi there,
If you start cassandra-cli with --debug it will output a stack trace if
the error is client side.
A long is exactly 8 bytes: 5
java.lang.RuntimeException: A long is exactly 8 bytes: 5
at
org.apache.cassandra.cli.CliClient.
Dne 10.10.2011 18:53, Mohit Anchlia napsal(a):
Does it mean you are not updating a row or deleting them?
yes. i have 350m rows and only about 100k of them are updated.
Can you look at JMX values of
BloomFilter* ?
i could not find this in jconsole mbeans or in jmx over http in
cassandra 1.0
on all nodes in cluster cassandra-0.8.5
[default@rapidshare] show schema;
A long is exactly 8 bytes: 5
how to capture more information for bug report?
Continuing this conversation. If there was a long running compaction
happening, I have to kill the node and start it again. Will it pick up
that compaction immediately?
no
Dne 10.10.2011 18:31, Yang napsal(a):
I noticed that 2 of my CFs are showing very different bloom filter
false ratios, one is close to 1.0;
the other one is only 0.3
cassandra bloom filters are computed for 1% false positive ratio.
is there any measure to increase the effectiveness of bloom fil
Dne 7.10.2011 23:16, Mohit Anchlia napsal(a):
You'll see output like:
Offset SSTables
1 8021
2 783
Which means 783 read operations accessed 2 SSTables
thank you for explaining it to me. I see this:
Offset SSTables
1 59323
2
Dne 7.10.2011 15:55, Mohit Anchlia napsal(a):
Check your disk utilization using iostat. Also, check if compactions
are causing reads to be slow. Check GC too.
You can look at cfhistograms output or post it here.
i dont know how to interpret cf historgrams. can you write it to wiki?
Dne 7.10.2011 10:04, aaron morton napsal(a):
Of the top of my head I it's not exposed via nodetool.
You can get it via HTTP if you install mx4j or if you could try
http://wiki.cyclopsgroup.org/jmxterm
i have MX4J/Http but cant find that info in listing.
i suspect that bloom filter performan
Dne 16.9.2011 8:20, Yang napsal(a):
I looked at the JMX attributes
CFS.BloomFilterFalseRatio, it's 1.0 , BloomFilterFalsePositives, it's
2810,
its possible to query this bloom filter false ratio from command line?
i have question for HOM FAQ:
http://rantav.github.com/hector/build/html/content/HOM/hector-object-mapper.html#hom-faq
please write short example how to persist objects in batch.
Dne 4.10.2011 22:05, Sorin Julean napsal(a):
Sorry for not being clear.
Indeed I mean a separate LAN and interfaces for "listen_address".
It needs to be 1GBit LAN, 100Mbit Ethernet is way too slow for cassandra.
simple script for running cluster - wide repairs
#! /bin/sh
set -e
if test $# -eq 0; then
echo "$0 "
exit 1;
fi
for i in `nodetool -h $1 ring | cut -d ' ' -f 1 | grep -e '^[0-9]'`; do
nodetool -h $i repair
done
1.
jamm still do not works on openJDK?
WARN [MutationStage:368] 2011-10-01 18:21:39,695 Memtable.java (line
156) MemoryMeter uninitialized (jamm not specified as java agent);
assuming liveRatio of 10.0. Usually this means cassandra-env.sh
disabled jamm because you are using a buggy JRE; upgr
bad news. using old hector breaks cassandra and it no longer boots.
ERROR [main] 2011-09-30 22:31:43,914 AbstractCassandraDaemon.java (line
354) Exc
eption encountered during startup
java.lang.IllegalArgumentException: value already present: 0
at
com.google.common.base.Preconditions.ch
also some exceptions on cassandra side:
Caused by: java.lang.IllegalArgumentException: value already present: 0
at
com.google.common.base.Preconditions.checkArgument(Preconditions.java
:115)
at
com.google.common.collect.AbstractBiMap.putInBothMaps(AbstractBiMap.j
ava:111)
Caught:
me.prettyprint.hector.api.exceptions.HCassandraInternalException: Cassan
dra encountered an internal error processing this request:
TApplicationError typ
e: 6 message:Internal error processing system_add_column_family
me.prettyprint.hector.api.exceptions.HCassandraInternalException:
Cas
is there hector snapshot available for 1.0rc? Hector 0.8 do not works
with it
> I have cleaned up my code that imports CSV into Cassandra and I have
put it open on https://github.com/nehalmehta/CSV2Cassandra. Have a look
if it is useful to you.
Hello,
I will remake this tool into something which is like Oracle
SQL*Loader. Basically, you will pass controlfile as command
Dne 26.9.2011 16:37, Jonathan Ellis napsal(a):
The seed names should match what the seeds advertise as
listen_address. I can't think of a reason host names shouldn't work,
I used DNS alias, that was probably reason why it didn't worked.
Dne 25.9.2011 22:40, aaron morton napsal(a):
That message will be logged if there RF on the keyspace is 1 or if the other
nodes are not up.
What's the RF ?
rf is 1.
Dne 25.9.2011 14:31, Radim Kolar napsal(a):
Dne 25.9.2011 9:29, Philippe napsal(a):
I have this happening on 0.8.x It looks to me as this happens when
the node is under heavy load such as unthrottled compactions or a
huge GC.
i have this problem too. Node down detection must be improved
Dne 25.9.2011 9:29, Philippe napsal(a):
I have this happening on 0.8.x It looks to me as this happens when the
node is under heavy load such as unthrottled compactions or a huge GC.
i have this problem too. Node down detection must be improved -
increased timeouts a bit or make more tries before
If you join a node with auto_bootstrap=false you had better be working
at quorum or higher to avoid stale/not found reads. You should then
repair the node right away to get all the missing data back on the
node. This is not suggested. It is best to leave auto_boostrap=true
and let Cassandra h
I just discovered that using host names for seed nodes in cassandra.yaml
do not work. This is done on purpose?
Dne 24.9.2011 0:05, Jonathan Ellis napsal(a):
Really large messages are not encouraged because they will fragment
your heap quickly. Other than that, no.
what is recommended chunk size for storing multi gigabyte files in
cassandra? 64MB is okay or its too large?
Dne 10.9.2011 21:48, Chris Goffinet napsal(a):
For things like rolling restarts, we do:
disablethrift
disablegossip
(...wait for all nodes to see this node go down..)
drain
I discovered problem with this advice.
If i do nodetool drain before killing node nodetool returns just after
flush and
Dne 22.9.2011 19:25, Nehal Mehta napsal(a):
We are trying to carry out same stuff, but instead of migrating into
JSON, we are exporting into CSV and than importing CSV into Cassandra.
You are right CSV seems to be more portable
Which DB are you currently using?
Postgresql and Apache Derby.
I need tool which is able to dump tables via JDBC into JSON format for
cassandra import. I am pretty sure that somebody already wrote that.
Are there tools which can do direct JDBC -> cassandra import?
Dne 21.9.2011 20:01, Jonathan Ellis napsal(a):
Means Cassandra couldn't create an empty file in the data directory
designating a sstable as compacted. I'd look for permissions
problems.
Short term there is no dire consequence, although it will keep
re-compacting that sstable. Longer term you'l
Sam,
thank you for your detailed problem description. What is reason why
delete cant remove old counter value from memtable? Because currently we
need to code workaround in our applications.
It would be nice to copy your description of this problem to:
http://wiki.apache.org/cassandra/Counter
Dne 21.9.2011 14:44, David Boxenhorn napsal(a):
The reason why counters work is that addition is commutative, but
deletes are not commutative
This is not my case. if you look at my 2 posts.
1st post seems to be CLI bug because same operation from program
works fine.
In 2nd post i alrea
Dne 21.9.2011 12:07, aaron morton napsal(a):
see technical limitations for deleting counters
http://wiki.apache.org/cassandra/Counters
For instance, if you issue very quickly the sequence "increment, remove,
increment" it is possible for the removal to be lost (if for some reason
the remove hap
Another problem with counters: Counter delete + incr does not set value
to 1 but to old value before deletion + 1
[default@whois] list ipbans;
Using default limit of 100
---
RowKey: 10.0.0.7
1 Row Returned.
[default@whois] incr ipbans['10.0.0.7']['hits'];
Value incremented.
[def
I cant get counter from CLI deleted:
[default@whois] get ipbans['10.0.0.7'];
=> (counter=hits, value=18)
Returned 1 results.
[default@whois] del ipbans['10.0.0.7'];
null
[default@whois] get ipbans['10.0.0.7'];
=> (counter=hits, value=18)
Returned 1 results.
[default@whois]
Is there good reason why cassandra is returning empty rows (no columns)
to client on rangeslicequery? (list command in CLI). They should be
hidden from user because it confuses applications.
They are not returned if get CLI command is used.
flush of Memtable-idx_graphable@915643571*(4671498/96780112
serialized/live bytes, 59891 ops)
can somebody explain why difference between live and serialized is so
big? Serialized bytes are actual bytes written to sstable?
*
If i add node without bootstrap and run repair on both nodes then no
data are moved to new node, just new writes are going to new node. This
is expected behaviour or it is some kind of user or program error?
101 - 200 of 208 matches
Mail list logo