When a compaction need to write a file cassandra will try to find a place to
put the new file, based on an estimate of it's size. If it cannot find enough
space it will trigger a GC which will delete any previously compacted and so
unneeded SSTables. The same thing will happen when a new SSTable
On 03/30/11 00:31, Peter Schuller wrote:
>
> set -e # important
> touch /path/to/flagfile.tmp
> nodetool -h localhost repair
> mv /path/to/flagfile.tmp /path/to/flagfile
>
Note this script doesn't work if your repair takes hours, and in the
middle of the repair cassandra was restarted, node
On 30/03/2011 09:08, aaron morton wrote:
Also as far as I understand we cannot immediately delete files because
other operations (including repair) may be using them. The data in the
pre compacted files is just as correct as the data in the compacted
file, it's just more compact. So the easiest
Hi.
Thanks a lot for your help.
One of the problems I have is that no-one here has clarified how
important this data is. I'm working on the assumption that it's
'somewhat important', but not critical data (nothing financial or
transactional).
So I don't need quorum; in fact as it will be a read/w
Hi all,
I've executed from cassandra-cli on a production server (0.7.0) the
following instruction:
update column family CFName with gc_grace=0;
I'm really surprise that Cassandra take a snapshots of the data for every
update that I executed.
Is this a normal behaviour?
Thanks,
Roberto Bentivogli
Checked the code again, got it a bit wrong. When getting a path to flush a
memtable (and to write an incoming stream) via cfs.getFlushPath() the code does
not invoke GC if there is not enough space.
One reason for not doing this could be that when we do it during compaction we
wait for 20 seco
AFAIK snapshots are taken for drop CF, drop KS and truncate CF also during
compaction if enabled via yaml. Cannot see anything in the code for 0.7 to
snapshot during CF update.
What was the name of the snapshot folder ?
btw it's a really good idea to upgrade to 0.7.4 if possible
Aaron
On 30
Hello,
We are using pycassa with uwsgi. There are about 16 uwsgi processes.
It is strange that one process can get the data which is queried by
another process. e.g. there is one row in column family A, looks like:
{row_key, {'column_a': 1, 'column_b': 2}}
process 1 run: get(row_key, columns=['
Hi,
Is there any way I can get multiple unique time UUIDs for the same timestamp
value - I mean, the UUIDs that are same in their time (most significant
bits), but differ in their least significant bits?
The least significant bits added by *
me.prettyprint.cassandra.utils.TimeUUIDUtils* seem to b
On 03/29/2011 01:18 PM, Peter Schuller wrote:
> (What *would* be useful perhaps is to be able to ask a node for the
> time of its most recently started repair, to facilitate easier
> comparison with GCGraceSeconds for monitoring purposes.)
I concur. JIRA time?
(Perhaps keeping track of the same
Hi!
I would like to query all the rows having a specific column defined, with
Hector
For example:
- CF is a column family
- rows 1 and 3 contain columns A and B
- rows 2 and 4 contain column A only
as a result of a query "column A" I would like to get rows 1-4, with column
A inside
as a result of
Hi Aaron,
I already deleted the snapshot folder unfortunately.
We upgraded to 0.7.4.
Thanks,
Roberto
On 30 March 2011 12:57, aaron morton wrote:
> AFAIK snapshots are taken for drop CF, drop KS and truncate CF also during
> compaction if enabled via yaml. Cannot see anything in the code for 0.7
> Note this script doesn't work if your repair takes hours, and in the
> middle of the repair cassandra was restarted, nodetool will exit and the
> flagfile will be updated. Another case, if repair hangs, and day later
> cassandra is restarted.
This is why "set -e" is at the to and commented as
Hi Roshan,
You probably want to look at Twitter's Snowflake:
https://github.com/twitter/snowflake
There's also another Java variant: https://github.com/earnstone/eid
- Drew
On Mar 30, 2011, at 6:08 AM, Roshan Dawrani wrote:
> Hi,
>
> Is there any way I can get multiple unique time UUIDs for
On Wed, Mar 30, 2011 at 12:54 PM, Peter Schuller
wrote:
>> Note this script doesn't work if your repair takes hours, and in the
>> middle of the repair cassandra was restarted, nodetool will exit and the
>> flagfile will be updated. Another case, if repair hangs, and day later
>> cassandra is re
> I concur. JIRA time?
https://issues.apache.org/jira/browse/CASSANDRA-2405
--
/ Peter Schuller
> I just wanted to chime in here and say some people NEVER run repair.
Just so long as the OP is understanding that this implies taking an
explicit decision to accept the "misbehavior" you will see as a
result. I.e., the reason people survive not doing repairs in some
cases is, as in your case, th
Hello,
I'm using CassandraBulkLoader.java
(https://svn.apache.org/repos/asf/cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java)
as base to build a bulk loader. Everything works fine for standard column
families but it fails when trying to insert data in super columns throwing this
ex
Hi All,
I have 6 node cassandra cluster all are setup with same
configurationI am getting fatal exceptions in one of the nodes
ERROR [Thread-604] 2011-03-29 20:19:13,218 AbstractCassandraDaemon.java
(line 114) Fatal exception in thread Thread[Thread-604,5,main]
java.lang.OutOfMemoryE
> I have 6 node cassandra cluster all are setup with same
> configuration I am getting fatal exceptions in one of the nodes
> ERROR [Thread-604] 2011-03-29 20:19:13,218 AbstractCassandraDaemon.java
> (line 114) Fatal exception in thread Thread[Thread-604,5,main]
> java.lang.OutOfMemor
Hi Everyone,
Anyone on this list interested in a remote very flexible contract gig? If yes,
please contact me directly.
Thanks,
Drew Kutcharian
Chief Technology Officer
Venarc Inc. www.venarc.com
Phone: 818-524-2500
http://wiki.apache.org/cassandra/MemtableThresholds#JVM_Heap_Size
On Wed, Mar 30, 2011 at 11:41 AM, Peter Schuller <
peter.schul...@infidyne.com> wrote:
> >I have 6 node cassandra cluster all are setup with same
> > configurationI am getting fatal exceptions in one of the nodes
> There's a wiki page somewhere that describes the overall rule of thumb
> for heap sizing, but I can't find it right now.
http://www.datastax.com/docs/0.7/operations/tuning#heap-sizing
--
/ Peter Schuller
I think I found a bug in the cassandra PHP client. I'm using phpcassa, but
the bug is in thrift itself, which I think that library phpcassa just
wraps. In any case, I was trying to test on my local machine, which has
limited RAM, so I reduced the JVM heap size. Of course I immediately had an
OOM
Nevermind, the header of the file says it's an apache project, so I'll
contact them. Though, if anyone else is running PHP and is worried about
dropped connections thrashing their server, apply this patch :-)
On Wed, Mar 30, 2011 at 3:18 PM, William Oberman
wrote:
> I think I found a bug in the
Are you looking at Thrift trunk, the thrift package that ships with phpcassa
0.7.a.3, or the phpcassa master branch?
On Wed, Mar 30, 2011 at 2:26 PM, William Oberman
wrote:
> Nevermind, the header of the file says it's an apache project, so I'll
> contact them. Though, if anyone else is running
Sorry, 0.7.a.3, I was just checking now if my issue still applies in either
new thrift or later phpcassa's. Probably jumped the gun on emailing this.
On Wed, Mar 30, 2011 at 3:28 PM, Tyler Hobbs wrote:
> Are you looking at Thrift trunk, the thrift package that ships with
> phpcassa 0.7.a.3, or
Actually, it looks like you're probably looking at the bundled Thrift in
phpcassa 0.7.a.3. This issue is fixed in Thrift trunk and the phpcassa
master branch.
On Wed, Mar 30, 2011 at 2:28 PM, Tyler Hobbs wrote:
> Are you looking at Thrift trunk, the thrift package that ships with
> phpcassa 0.7
Well, I'll wait for the next official release then. Sorry for the
confusion!
On Wed, Mar 30, 2011 at 3:31 PM, Tyler Hobbs wrote:
> Actually, it looks like you're probably looking at the bundled Thrift in
> phpcassa 0.7.a.3. This issue is fixed in Thrift trunk and the phpcassa
> master branch.
No worries! I think everybody here would rather hear a duplicate report of
an issue than never hear about it at all.
On Wed, Mar 30, 2011 at 2:39 PM, William Oberman
wrote:
> Well, I'll wait for the next official release then. Sorry for the
> confusion!
>
>
> On Wed, Mar 30, 2011 at 3:31 PM, Ty
I'm a little unclear on the differences between the nodetool operations:
- compaction
- repair
- clean
I understand that compaction consolidates the SSTables and physically performs
deletes by taking into account the Tombstones. But what does clean and repair
do then?
This bug was fixed in thrift php trunk
2011/3/30 William Oberman
> I think I found a bug in the cassandra PHP client. I'm using phpcassa, but
> the bug is in thrift itself, which I think that library phpcassa just
> wraps. In any case, I was trying to test on my local machine, which has
> limi
Does the following load balancing scenario look reasonable with cassandra ?
I will not be having any app servers.
http://dl.dropbox.com/u/7258508/2011-03-30_1542.png
Thanks.
Short answers:
- compaction - Initiate immediate full compaction. Removes deleted data.
- clean - Initiates immediate cleanup i.e. remove the data that is deleted
and that doesn't belong to this node. Internally performs full compaction.
- repair - Used to make different copies (replicas) of data c
Unsubscribe
Sent from my iPhone
On 30.3.2011, at 22.39, William Oberman wrote:
> Well, I'll wait for the next official release then. Sorry for the confusion!
>
> On Wed, Mar 30, 2011 at 3:31 PM, Tyler Hobbs wrote:
> Actually, it looks like you're probably looking at the bundled Thrift in
>
I did an apple to apple comparison of WRITE speed of Mongo vs Cassandra.
Benchmarking (Cluster of 4 machines. Each of 4 cores and 15gb ram. 2
IN region1 but different zones, 1 in region2 and 1 in region3. Set
consistency level of 3. i.e. atleast 3 machines have to confirm of
write before the write
My concern when I see something like this is it might cause developers
on the project to get worried and start to try to solve the wrong
problems. Cassandra is not going to be as easy as Mongo, certainly
not any time soon. CQL won't do it, although it will help. This
isn't a criticism of Cassand
After restarting a Cassandra 0.7.2 node, the node catches an exception
during initialization and refuses to start:
Caused by: org.apache.cassandra.config.ConfigurationException: Attempt
to assign id to existing column family.
at org.apache.cassandra.config.CFMetaData.map(CFMetaData.jav
On Wed, Mar 30, 2011 at 3:15 AM, Eric Evans wrote:
> The client space as a whole *is* a mess, despite heroic efforts on the
> part of our third-party API maintainers, but forcing them in-tree is not
> going to solve anything. In fact, it would very likely make it worse by
> adding unnecessary ov
If I understand the question, it's not that
UUIDGen.makeType1UUIDFromHost(InetAddress.getLocalHost()) is returning
duplicate UUID's. It should always be giving unique time-based uuids
and has checks to make sure it does. The question was whether it was
possible to get multiple unique time-based U
On Wed, Mar 30, 2011 at 2:53 AM, Drew Kutcharian wrote:
> Hi Gregori,
>
> I'm about to start a new project and I was considering using MongoDB too,
> but I just couldn't find a nice way to scale it. Seems like for scaling you
> need to use the same style as MySQL, having master/slaves and replica
Hello Edward,
have you a working kudera+spring example with JPA annotations?
Best Regards,
Anton
While I respect your decision...
If you are tired of writing code there are solutions around coding
everything there are tools like http://code.google.com/p/kundera/
This is verbose:
http://java
Thanks for the feedback Gregori,
We in Australia are only concerned with solutions as we are a solutions focused
organization. With respect to your feedback, you and your team seem to have
identified no solutions other than jumping ship. When we subscribed to the 50
or so emails per day, we wan
On 03/30/2011 02:54 PM, Jeremy Stribling wrote:
After restarting a Cassandra 0.7.2 node, the node catches an exception
during initialization and refuses to start:
Caused by: org.apache.cassandra.config.ConfigurationException: Attempt
to assign id to existing column family.
at
org.apac
Hi Ed,
There's no need to re-invent the wheel that's pretty much what Twitter
Snowflake does. The way it works is it creates a 64 bit long id which is
formatted as such
time_bits : data_center_id : machine_id : sequence
Where time_bits are the milliseconds since a custom epoch.
So If you see,
I am using 16G of heap space how much more should i increase.
Please suggest
Thanks
Anurag
On Wed, Mar 30, 2011 at 11:43 AM, Narendra Sharma wrote:
> http://wiki.apache.org/cassandra/MemtableThresholds#JVM_Heap_Size
>
>
>
> On Wed, Mar 30, 2011 at 11:41 AM, Peter Schuller <
> peter.schul...@infi
Hi All,
I am getting following message from cassandra
WARN [CompactionExecutor:1] 2011-03-30 18:46:33,272 CompactionManager.java
(line 406) insufficient space to compact all requested files SSTableReader(
I am using 16G of java heap space ,please let me know should I consider this
as a
OOM at startup with 16GB... seems like an issue. Which version are you
using? Can you provide some details on "failed node"? What exactly happened?
That might give some clue. Also, you might want to start with log level set
to debug to findout what more on what exactly Cassandra is doing that is
ca
The space referred in the log message is disk space and not heap. So check
if you are running low on disk space.
Thanks,
Naren
On Wed, Mar 30, 2011 at 4:55 PM, Anurag Gujral wrote:
> Hi All,
> I am getting following message from cassandra
>
> WARN [CompactionExecutor:1] 2011-03-30 18:
Hey Drew, I'm somewhat familiar with Snowflake, and it's certainly a
good option, but, my impression was that the main reason to use it is
because you find the 128-bits for a UUID overkill, not because it's
doing anything you can't do with UUID's. The difference in time
resolution between UUIDs an
I restarted the cassandra node with more disks when I try to connect to
cassandra i get connection refused errors.
I see the following in the logs
INFO [HintedHandoff:1] 2011-03-30 19:37:42,189 HintedHandOffManager.java
(line 304) Started hinted handoff for endpoint /10.97.145.35
INFO [HintedHando
I am using version 0.7.3 I am not sure what happened I tried connecting to
the node and when I could not connect I looked at error logs i saw the log
messages which I pasted below
On Wed, Mar 30, 2011 at 5:04 PM, Narendra Sharma
wrote:
> OOM at startup with 16GB... seems like an issue. Which ver
Hi Everyone,
I just read Jonathan Ellis' great post on Secondary Indexes
(http://www.datastax.com/dev/blog/whats-new-cassandra-07-secondary-indexes) and
I was wondering where I can find a bit more info on them. I would like to know:
1) Are there in limitations beside the hash properties (no be
Hi Ed,
Cool, I guess we both read/interpreted his post differently and gave two valid
answers ;)
- Drew
On Mar 30, 2011, at 5:40 PM, Ed Anuff wrote:
> Hey Drew, I'm somewhat familiar with Snowflake, and it's certainly a
> good option, but, my impression was that the main reason to use it is
>
Hello,
I'm trying to build and run cassandra 0.7.4-src on IntelliJ IDEA 10 CE
on OSX with reading
http://wiki.apache.org/cassandra/RunningCassandraInIDEA.
Though I need to omit interface/avro/gen-java, exclude
java/org/apache/cassandra/hadoop, and
download&add jna.jar into library path, I could ki
Woud you cassandra team think to add an alias name for nodetool
"repair" command?
I mean, the word "repair" scares some of people.
When I say "we need to run nodetool repair regularly on cassandra
nodes", they think "OH... Those are broken so often!".
So if I can say it in more soft word, ex. "syn
Hi All,
I had a single directory designated as data file directory in
cassandra ,but it ran out of space .I am adding a new data directory in
addition to the one I have .What should I do so that cassandra balance the
space on both of them.
Thanks
Anurag
It really helps. Thank you very much.
Sheng
2011/3/30 aaron morton
> When a compaction need to write a file cassandra will try to find a place
> to put the new file, based on an estimate of it's size. If it cannot find
> enough space it will trigger a GC which will delete any previously compact
I just found an estmateKeys() method of the ColumnFamilyStoreMBean.
Is there any indication about how it works?
Sheng
2011/3/28 Sheng Chen
> Hi all,
> I want to know how many records I am holding in Cassandra, just like
> count(*) in sql.
> What can I do ? Thank you.
>
> Sheng
>
>
>
On 3/30/2011 12:39 PM, aaron morton wrote:
Checked the code again, got it a bit wrong. When getting a path to
flush a memtable (and to write an incoming stream) via
cfs.getFlushPath() the code does not invoke GC if there is not enough
space.
One reason for not doing this could be that when we
60 matches
Mail list logo