Hi,
Currently both secondary index build and compactions are run by compaction
manager within the same thread pool (C* 2.0.x). I'm currently experiencing
a situation where repair's stream sessions get stuck waiting because they
don't get scheduled to build secondary indexes (and finalise the
Thanks Andrey. Also found this ticket regarding this issue:
https://issues.apache.org/jira/browse/CASSANDRA-2698
On Tue, Oct 16, 2012 at 8:00 PM, Andrey Ilinykh ailin...@gmail.com wrote:
In my experience running repair on some counter data, the size of
streamed data is much bigger than the
I think this JIRA answers your question:
https://issues.apache.org/jira/browse/CASSANDRA-2610
which in order not to duplicate work (creation of Merkle trees) repair
is done on all replicas for a range.
Cheers,
Omid
On Tue, Sep 25, 2012 at 8:27 AM, Sergey Tryuber stryu...@gmail.com wrote:
Hi
On Thu, Sep 13, 2012 at 1:43 AM, Bryan Talbot btal...@aeriagames.com wrote:
To answer my own question: yes, the error is fatal. This also means that
upgrades to 1.1.x from 1.0.x MUST use 1.0.11 or greater it seems to be
successful.
My test upgrade from 1.0.9 to 1.1.5 left the cluster in a
On Wed, Sep 12, 2012 at 9:38 AM, Janne Jalkanen
janne.jalka...@ecyrd.com wrote:
OK, so what's the worst case here? Data loss? Bad performance?
Low performance is for sure a side effect. I can't comment on data
loss (and I'm curious about as well) because it depends on how data
off of an
Which version of Cassandra has your data been created initially with?
A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables
and inter-level overlaps in CFs with Leveled Compaction. Your sstables
generated with 1.1.3 and later should not have this issue [1] [2].
In case you have
Could you, as Aaron suggested, open a ticket?
-- Omid
On Tue, Sep 11, 2012 at 2:35 PM, Rudolf van der Leeden
rudolf.vanderlee...@scoreloop.com wrote:
Which version of Cassandra has your data been created initially with?
A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables
On Tue, Sep 11, 2012 at 8:33 PM, Janne Jalkanen
janne.jalka...@ecyrd.com wrote:
A bug in Cassandra 1.1.2 and earlier could cause out-of-order sstables
and inter-level overlaps in CFs with Leveled Compaction. Your sstables
generated with 1.1.3 and later should not have this issue [1] [2].
Do you see exceptions like java.lang.UnsupportedOperationException:
Not a time-based UUID in log files of nodes running 1.0.6 and 1.0.9?
Then it's probably due to [1] explained here [2] -- In this case you
either have to upgrade all nodes to 1.1.4 or if you prefer keeping a
mixed-version cluster,
The change is introduced in 1.1.1 [1] which takes replication factor
into account [2]. That's why 18.75% * 16 = 300% (which you have
replication factor of 3).
[1] https://issues.apache.org/jira/browse/CASSANDRA-3412
[2]
Secondly, what's the need for sleep 120?
just give the cluster a chance to settle down between repairs...
there's no real need for it, just is there because.
Actually, repair could cause unreplicated data to be streamed and new
sstables to be created. New sstables could cause pending
It works for me on Sun's jvm. It appears to be similar to:
http://www-01.ibm.com/support/docview.wss?uid=swg1IV12627
Have you tried upgrading IBM's java or using Sun's?
-- Omid
On Mon, Aug 13, 2012 at 1:34 PM, Robert Hellmans robert.hellm...@aastra.com
wrote:
**
Hi
I'm trying the
I think the 1.0.6 nodes can't join (on a restart, for example) the
ring in a mixed cluster. So you can either continue the rolling
upgrade, or if you want to keep running in mixed mode (why?) you can
use 1.0.11 for the older ones, but you won't be able to do schema
migrations.
-- Omid
On Fri,
Hey,
It is explained here:
https://issues.apache.org/jira/browse/CASSANDRA-4195
-- Omid
On Wed, Aug 1, 2012 at 2:39 AM, Roshan codeva...@gmail.com wrote:
Hi
I have 3 node development cluster and all running 1.0.6 version without any
issue. As the part of the upgrade to 1.1.2, I just
Hey,
Mutations taking longer than rpc_timeout will be dropped because
coordinator won't be waiting for the coordinator and will return
TimeoutException to the client, if it doesn't reach the consistency level
[1].
In case of counters though, since counter mutations aren't idempotent, the
client
Hi,
I'm trying to tune, memtable size, key cache size and heap size on
Cassandra 1.1.0 but I keep having memory pressure and reduced cache size.
With the following settings:
heap size: 10GB (had the same issue with 8GB so I'm testing with increased
heap size)
memtable_total_space_in_mb: 2GB
I think the change log for 1.1.1 is missing CASSANDRA-4150 [1].
-- Omid
[1] https://issues.apache.org/jira/browse/CASSANDRA-4150
On Mon, Jun 4, 2012 at 9:52 PM, Sylvain Lebresne sylv...@datastax.com wrote:
The Cassandra team is pleased to announce the release of Apache Cassandra
version
Also looks similar to this ticket:
https://issues.apache.org/jira/browse/CASSANDRA-4078https://issues.apache.org/jira/browse/CASSANDRA-4078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
On Thu, Jun 7, 2012 at 6:48 PM, Omid Aladini omidalad...@gmail.com wrote:
Hi,
One
.
--
Sylvain
On Fri, Jun 8, 2012 at 12:06 PM, Omid Aladini omidalad...@gmail.com
wrote:
Also looks similar to this ticket:
https://issues.apache.org/jira/browse/CASSANDRA-4078
On Thu, Jun 7, 2012 at 6:48 PM, Omid Aladini omidalad...@gmail.com
wrote:
Hi,
One of my 1.1.1 nodes doesn't
Hi,
One of my 1.1.1 nodes doesn't restart due to stack overflow on building the
interval tree. Bumping the stack size doesn't help. Here's the stack trace:
https://gist.github.com/2889611
It looks more like an infinite loop on IntervalNode constructor's logic
than a deep tree since DEBUG log
-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com
On 25/04/2012, at 1:11 AM, Omid Aladini wrote:
Hi,
I'm experiencing losing a part of key cache on restart on Cassandra 1.0.7.
For example:
- cfstats reports key cache size of 13,040,502
Hi,
I'm experiencing losing a part of key cache on restart on Cassandra 1.0.7.
For example:
- cfstats reports key cache size of 13,040,502 with capacity of 15,000,000.
- Cassandra log reports 12,955,585 of them have been saved on the last save
events.
- On restart Cassandra reads saved cache.
-
Hi,
Cassandra issues an mlockall [1] before mmap-ing sstables to prevent
the kernel from paging out heap space in favor of memory-mapped
sstables. I was wondering, what happens to the off-heap row cache
(saved or unsaved)? Is it possible that the kernel pages out off-heap
row cache in favor of
2012/4/5 Omid Aladini omidalad...@gmail.com
Hi,
I'm experiencing a steady growth in resident size of JVM running
Cassandra 1.0.7. I disabled JNA and off-heap row cache, tested with
and without mlockall disabling paging, and upgraded to JRE 1.6.0_31 to
prevent this bug [1] to leak memory
Hi,
I'm experiencing a steady growth in resident size of JVM running
Cassandra 1.0.7. I disabled JNA and off-heap row cache, tested with
and without mlockall disabling paging, and upgraded to JRE 1.6.0_31 to
prevent this bug [1] to leak memory. Still JVM's resident set size
grows steadily. A
25 matches
Mail list logo