Well, just did not look at these logs very well at all last night
First out of disk message:
ERROR [CompactionExecutor:387] 2011-05-02 01:16:01,027
AbstractCassandraDaemon.java (line 112) Fatal exception in thread
Thread[CompactionExecutor:387,1,main]
java.io.IOException: No space left on device
T
Hi Aaron
It has no data files whatsoever. The upgrade path is 0.7.4 -> 0.7.5. It
turns out the initial problem was the sw raid failing silently because
of another faulty disk.
Now that the storage is working, I brought up the node again, same IP,
same token and tried doing nodetool repair.
All
Hi all,
I ran decommission on a node in my 32 node cluster. After about an hour of
streaming files to another node, I got this error on the node being
decommissioned:
INFO [MiscStage:1] 2011-05-03 21:49:00,235 StreamReplyVerbHandler.java (line
58) Need to re-stream file /raiddrive/MDR/MeterReco
Thanks Peter.
I believe...I found the root cause. Switch that we used was bad.
Now on a 4 node cluster ( Each Node has 1 CPU - Quad Core and 16 GB of RAM),
I was able to get around 11,000 writes and 10,050 reads per second
simultaneously (CPU usage is around 45% on all nodes. Disk queue size
Hm... peculiar.
Post flush is not involved in compactions, right?
May 2nd
01:06 - Out of disk
01:51 - Starts a mix of major and minor compactions on different column
families
It then starts a few minor compactions extra over the day, but given that
there are more than 1000 sstables, and we are ta
Snapshot runs on a local node. How do I ensure I have a 'point in
time' snapshot of the full cluster ? Do I have to stop the writes on
the full cluster and then snapshot all the nodes individually ?
Thanks.
On Tue, 2011-05-03 at 14:22 -0500, Jonathan Ellis wrote:
> Can you create a ticket?
CASSANDRA-2598
On Tue, May 3, 2011 at 4:08 PM, Jonathan Ellis wrote:
> On Tue, May 3, 2011 at 2:46 PM, aaron morton wrote:
>> Jonathan,
>> I think you are saying each DC should have it's own (logical) token
>> ring.
>
> Right. (Only with NTS, although you'd usually end up with a similar
> effect if you
Can you provide some details of the data returned from you do the =
get_range() ? It will be interesting to see the raw bytes returned for =
the keys. The likely culprit is a change in the encoding. Can you also =
try to grab the bytes sent for the key when doing the single select that =
fails.=20
On Tue, May 3, 2011 at 2:46 PM, aaron morton wrote:
> Jonathan,
> I think you are saying each DC should have it's own (logical) token
> ring.
Right. (Only with NTS, although you'd usually end up with a similar
effect if you alternate DC locations for nodes in a ONTS cluster.)
> Bu
Compaction does, but flush didn't until
https://issues.apache.org/jira/browse/CASSANDRA-2404
On Tue, May 3, 2011 at 2:26 PM, Terje Marthinussen
wrote:
> Yes, I realize that.
> I am bit curious why it ran out of disk, or rather, why I have 200GB empty
> disk now, but unfortunately it seems like we
To give an idea, last March (2010) I run the a much older Cassandra on 10 HP
blades (dual socket, 4 core, 16GB, 2.5 laptop HDD) and was writing around 250K
columns per second with 500 python processes loading the data from wikipedia
running on another 10 HP blades.
This was my first out of the
When you say "it's clean" does that mean the node has no data files ?
After you replaced the disk what process did you use to recover ?
Also what version are you running and what's the recent upgrade history ?
Cheers
Aaron
On 3 May 2011, at 23:09, Héctor Izquierdo Seliva wrote:
> Hi everyone.
Jonathan,
I think you are saying each DC should have it's own (logical) token
ring. Which makes sense as the only way to balance the load in each dc. I think
most people assume (including me) there was a single token ring for the entire
cluster.
But currently two endpoints can
Yes, I realize that.
I am bit curious why it ran out of disk, or rather, why I have 200GB empty
disk now, but unfortunately it seems like we may not have had monitoring
enabled on this node to tell me what happened in terms of disk usage.
I also thought that compaction was supposed to resume (try
Ah, that makes sense. snapshot_before_compaction is trying to
snapshot, but incremental_backups already created one (for newly
flushed sstables). You're probably the only one running with both
options on. :)
Can you create a ticket?
On Tue, May 3, 2011 at 2:05 PM, Mck wrote:
> On Tue, 2011-05-
You're right, this is an oversight. Created
https://issues.apache.org/jira/browse/CASSANDRA-2596 to fix.
As for a workaround, you can drop the index + recreate. (Upgrade to
0.7.5 first, if you haven't yet.)
On Tue, May 3, 2011 at 3:22 AM, Arsene Lee
wrote:
> Hi,
>
>
>
> We are trying to use sna
On Tue, 2011-05-03 at 13:52 -0500, Jonathan Ellis wrote:
> you should probably look to see what errno 17 means for the link
> system call on your system.
That the file already exists.
It seems cassandra is trying to make the same hard link in parallel
(under heavy write load) ?
I see now i can a
you should probably look to see what errno 17 means for the link
system call on your system.
On Tue, May 3, 2011 at 9:52 AM, Mck wrote:
> Running a 3 node cluster with cassandra-0.8.0-beta1
>
> I'm seeing the first node logging many (thousands) times lines like
>
>
> Caused by: java.io.IOExceptio
post flusher is responsible for updating commitlog header after a
flush; each task waits for a specific flush to complete, then does its
thing.
so when you had a flush catastrophically fail, its corresponding
post-flush task will be stuck.
On Tue, May 3, 2011 at 1:20 PM, Terje Marthinussen
wrote
On Tue, 2011-05-03 at 16:52 +0200, Mck wrote:
> Running a 3 node cluster with cassandra-0.8.0-beta1
>
> I'm seeing the first node logging many (thousands) times
Only "special" thing about this first node is it receives all the writes
from our sybase->cassandra import job.
This process migrates
So yes, there is currently some 200GB empty disk.
On Wed, May 4, 2011 at 3:20 AM, Terje Marthinussen
wrote:
> Just some very tiny amount of writes in the background here (some hints
> spooled up on another node slowly coming in).
> No new data.
>
> I thought there was no exceptions, but I did not
Just some very tiny amount of writes in the background here (some hints
spooled up on another node slowly coming in).
No new data.
I thought there was no exceptions, but I did not look far enough back in the
log at first.
Going back a bit further now however, I see that about 50 hours ago:
ERROR
Both data and system load are equal across all nodes and the smaller test
cluster also exhibits the same issue. tokens are balanced and total node size
is equivalent.
On May 3, 2011, at 10:51 AM, Jonathan Ellis wrote:
> So either (a) dynamic snitch is wrong or (b) those nodes really are
> more
... and are there any exceptions in the log?
On Tue, May 3, 2011 at 1:01 PM, Jonathan Ellis wrote:
> Does it resolve down to 0 eventually if you stop doing writes?
>
> On Tue, May 3, 2011 at 12:56 PM, Terje Marthinussen
> wrote:
>> Cassandra 0.8 beta trunk from about 1 week ago:
>> Pool Name
Does it resolve down to 0 eventually if you stop doing writes?
On Tue, May 3, 2011 at 12:56 PM, Terje Marthinussen
wrote:
> Cassandra 0.8 beta trunk from about 1 week ago:
> Pool Name Active Pending Completed
> ReadStage 0 0 5
Cassandra 0.8 beta trunk from about 1 week ago:
Pool NameActive Pending Completed
ReadStage 0 0 5
RequestResponseStage 0 0 87129
MutationStage 0 0 187298
ReadRe
So either (a) dynamic snitch is wrong or (b) those nodes really are
more heavily loaded than the others, and are correctly pushing queries
to other replicas.
On Tue, May 3, 2011 at 12:47 PM, Serediuk, Adam
wrote:
> I just ran a test and we do not see that behavior with dynamic snitch
> disabled.
I just ran a test and we do not see that behavior with dynamic snitch disabled.
All nodes appear to be doing local reads as expected.
On May 3, 2011, at 10:37 AM, Jonathan Ellis wrote:
> Do you still see this behavior if you disable dynamic snitch?
>
> On Tue, May 3, 2011 at 12:31 PM, Serediuk
Do you still see this behavior if you disable dynamic snitch?
On Tue, May 3, 2011 at 12:31 PM, Serediuk, Adam
wrote:
> We appear to have encountered an issue with cassandra 0.7.5 after upgrading
> from 0.7.2. While doing a batch read using a get_range_slice against the
> ranges an individual node
We appear to have encountered an issue with cassandra 0.7.5 after upgrading
from 0.7.2. While doing a batch read using a get_range_slice against the ranges
an individual node is master for we are able to reproduce consistently that the
last two nodes in the ring, regardless of the ring size (we
Writing to Cassandra from map/reduce jobs over HDFS shouldn't be a problem.
We're doing it in our cluster and I know of others doing the same thing. You
might just make sure the number of reducers (or mappers) writing to cassandra
don't overwhelm it. There's no data locality for writes, thoug
Did you do a bulk upload with mysql from the same machine or separate
insert/commit for each row? And did you run inserts from the same machine as
the mysqld server?
--
View this message in context:
http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/low-performance-inserting-tp63268
I would add that running one cluster is operationally less work than
running multiple.
On Tue, May 3, 2011 at 4:15 AM, David Boxenhorn wrote:
> If I have a database that partitions naturally into non-overlapping
> datasets, in which there are no references between datasets, where each
> dataset i
The way we solved this problem is that it turned out we had only a few
hundred rows with unicode keys, so we simply extracted them, upgraded to
0.7, and wrote them back. However, this means that among the rows, there are
a few hundred weird duplicate rows with identical keys.
Is this going to be a
Hey everyone,
We did some tests before upgrading our Cassandra cluster from 0.6 to 0.7,
just to make sure that the change in how keys are encoded wouldn't cause us
any dataloss. Unfortunately it seems that rows stored under a unicode key
couldn't be retrieved after the upgrade. We're running every
Hi Sylvain,
thanks for your answer.
I'd make a test with the stress utility inserting 100 000 rows with 10
columns per row
I use these options: -o insert -t 5 -n 10 -c 10 -d
192.168.1.210,192.168.1.211,...
result: 161 seconds
with MySQL using inserts (after a dump): 1.79 second
Charles
201
On Tue, May 3, 2011 at 10:13 AM, Jonathan Ellis wrote:
> Right, when you are computing balanced RP tokens for NTS you need to
> compute the tokens for each DC independently.
I am confused ... sorry. Are you saying that ... I need to change how
my keys are calculated to fix this problem? Or are
Running a 3 node cluster with cassandra-0.8.0-beta1
I'm seeing the first node logging many (thousands) times lines like
Caused by: java.io.IOException: Unable to create hard link
from
/iad/finn/countstatistics/cassandra-data/countstatisticsCount/thrift_no_finntech_countstats_count_Count_129947
So we are currently running a 10 node ring in one DC, and we are going to be
adding 5 more nodes
in another DC. To keep the rings in each DC balanced, should I really
calculate the tokens independently
and just make sure none of them are the same? Something like:
DC1 (RF 5):
1: 0
2: 1
You don't give many details, but I would guess:
- your benchmark is not multithreaded
- mongodb is not configured for durable writes, so you're really only
measuring the time for it to buffer it in memory
- you haven't loaded enough data to hit "mongo's index doesn't fit in
memory anymore"
On Tue
Right, when you are computing balanced RP tokens for NTS you need to
compute the tokens for each DC independently.
On Tue, May 3, 2011 at 6:23 AM, aaron morton wrote:
> I've been digging into this and worked was able to reproduce something, not
> sure if it's a fault and I can't work on it any m
There is probably a fair number of things you'd have to make sure you do to
improve the write performance on the Cassandra side (starting by using multiple
threads to do the insertion), but the first thing is probably to start
comparing things
that are at least mildly comparable. If you do inserts
If snapshot doesn't include secondary indexes then we can't use it for our
backup and restore procedure.
.
This mean, we need to stop our service when we want to do backups and this
would cause longer system down time.
If there is no particular reason, it is probably a good idea to also include
Use more nodes to increase your write throughput. Testing on a single
machine is not really a viable benchmark for what you can achieve with
cassandra.
Hi,
Not sure this is the case for your Bad Performance, but you are Meassuring Data
creation and Insertion together. Your Data creation involves Lots of class
casts which are probably quite Slow.
Try
Timing only the b.send Part and See how Long that Takes.
Roland
Am 03.05.2011 um 12:30 schrieb
I am working for client that needs to persist 100K-200K records per second
for later querying. As a proof of concept, we are looking at several
options including nosql (Cassandra and MongoDB).
I have been running some tests on my laptop (MacBook Pro, 4GB RAM, 2.66 GHz,
Dual Core/4 logical cores)
Looking at the code for the snapshot it looks like it does not include
secondary indexes. And I cannot see a way to manually trigger an index rebuild
(via CFS.buildSecondaryIndexes())
Looking at this it's probably handy to snapshot them
https://issues.apache.org/jira/browse/CASSANDRA-2470
I'm
I've been digging into this and worked was able to reproduce something, not
sure if it's a fault and I can't work on it any more tonight.
To reproduce:
- 2 node cluster on my mac book
- set the tokens as if they were nodes 3 and 4 in a 4 node cluster, e.g. node 1
with 8507059173023461586584365
Hi everyone. One of the nodes in my 6 node cluster died with disk
failures. I have replaced the disks, and it's clean. It has the same
configuration (same ip, same token).
When I try to restart the node it starts to throw mmap underflow
exceptions till it closes again.
I tried setting io to stand
Please consider the environment before printing this e-mail
Important - The information contained in this email (and any attached files) is
confidential and may be legally privileged and protected by law.
The intended recipient is authorised to access it. If you are not the intended
reci
Hello everybody,
first: sorry for my english in advance!!
I'm getting started with Cassandra on a 5 nodes cluster inserting data
with the pycassa API.
I've read everywere on internet that cassandra's performance are better than
MySQL
because of the writes append's only into commit logs files.
W
If I have a database that partitions naturally into non-overlapping
datasets, in which there are no references between datasets, where each
dataset is quite large (i.e. large enough to merit its own cluster from the
point of view of quantity of data), should I set up one cluster per database
or one
If your are still having problems can you say what version, how many nodes,
what RF, what CL and if after inserting and failing on the first get it works
on a subsequent get.
Thanks
Aaron
On 3 May 2011, at 18:54, chovatia jaydeep wrote:
> One small correction in my mail below.
> Second inse
Hi,
We are trying to use snapshot for backup and restore. We found out that
snapshot doesn't take secondary indexes.
We are wondering why is that? And is there any way we can rebuild the secondary
index?
Regards,
Arsene
Hi Jeremy,
yes, the setup on the data-nodes is:
- Hadoop DataNode
- Hadoop TaskTracker
- CassandraDaemon
However - the map-input is not read from Cassandra. I am running a writing
stress test - no reads (well from time to time I check the produced items using
cassandra
56 matches
Mail list logo