Mubarak Seyed apple.com> writes:
>
> - How does client (application) connect to cassandra cluster? Is it always for
one node (and thrift can get ring info) and send the request to connected node
This depends on client library you use. Any cassandra node can accept client
connections and forward
Hello,
I keep reading everywhere that Cassandra has supported multiple
datacenters from the beginning. I would like to know what does Cassandra
do to achieve that. Is it just that the developers have written some code
that supports that scenario, or is there something inherent in Cassandra's
desig
Hi,
I have noticed the following behaviour (bug?) which I don't completely
understand:
1. start Cassandra (I'm using 0.6.2, but it also appears in 0.6.1)
2. work with it (I'm using Java thrift API)
3. let it sit for a long time (in my case: a day or more) without
issuing any command
4. go back to
On Fri, 2010-06-18 at 12:27 +0530, Atul Gosain wrote:
> Is the client API for cassandra available in AVRO.
Significant parts of it, but it is not yet finished.
> If so, any links to examples or some documentation?
There is no samples or documentation yet, sorry.
> and If so, any comparison betw
Hi all,
@Chris, Did you get any bench you could share with us?
I am running the same kind of test on EC2 (m.large instances) :
- one VM for stress.py (can be launched several times)
- another VM for a unique cassandra node
I use the default conf settings (Xmx 1G, concurrentwrite 32...) except f
Our cassandra client fails over if a node times out. Aside from actual
failure, repair and major compactions can make a node so slow that it
affects application performance.
One problem we've run in to is that a node in the midst of repair will still
have requests routed to it internally, even if
Would it perhaps be worth denormalising your data so that you can
retrieve all rows as a single row using a key encoded with the query
predicate?
Until we get a stored proc feature (dunno if planned) it's hard to
avoid round trips without denormalizing/replication of data to fit
your quer
Assume a map/reduce program which needs to update some values during
ingest, and needs to perform read operations on 100 keys each of which
have say 50 different columns. This happens many times for a given
reduce task in the cluster. Shouldn't that be handled by the server
as a single call?
On
In 0.6 your only option with those constraints is to iterate over the
entire CF and deleting row by row. This requires you are either using
OPP or have an index that covers all keys in the CF. 0.7 adds the
ability to truncate a CF (deleting all its rows) through the API.
On Fri, Jun 18, 2010 at
Would be interesting to have a snitch that manipulated responses for
read nodes based on historical response times.
On Fri, Jun 18, 2010 at 8:21 AM, James Golick wrote:
> Our cassandra client fails over if a node times out. Aside from actual
> failure, repair and major compactions can make a node
See https://issues.apache.org/jira/browse/CASSANDRA-981
-Original Message-
From: "Benjamin Black"
Sent: Friday, June 18, 2010 12:32pm
To: user@cassandra.apache.org
Subject: Re: Failover and slow nodes
Would be interesting to have a snitch that manipulated responses for
read nodes based o
To summarize:
If a request for a column comes in *after a period of several hours
with no requests*, then the node servicing the request hangs while
looking for its peer rather than servicing the request like it should.
It then throws either a TimedOutException or a (wrong)
NotFoundExeption.
And
On 06/18/2010 01:20 AM, alta...@ceid.upatras.gr wrote:
I also read about an implemenetation of Rack Awareness employing
Zookeeper, but I gather that wasn't released by Facebook and it was more
geared towards single-DC rack awareness because Zookeeper is a bit heavy
on the bandwidth.
Bandwidth
On Fri, Jun 18, 2010 at 8:00 AM, Olivier Mallassi wrote:
> I use the default conf settings (Xmx 1G, concurrentwrite 32...) except for
> commitlog and DataFileDirectory : I have a raid0 EBS for commit log and
> another raid0 EBS for data.
> I can't get through 7500 write/sec (when launching 4 stres
Perfect, ship it.
On Fri, Jun 18, 2010 at 10:37 AM, Stu Hood wrote:
> See https://issues.apache.org/jira/browse/CASSANDRA-981
>
> -Original Message-
> From: "Benjamin Black"
> Sent: Friday, June 18, 2010 12:32pm
> To: user@cassandra.apache.org
> Subject: Re: Failover and slow nodes
>
> W
On Jun 18, 2010, at 8:01 AM, Eric Evans wrote:
> On Fri, 2010-06-18 at 12:27 +0530, Atul Gosain wrote:
>> Is the client API for cassandra available in AVRO.
> Significant parts of it, but it is not yet finished.
>> If so, any links to examples or some documentation?
> There is no samples or docume
I have been reminded that you can do a range query+pagination with RP
in 0.6 to perform this operation.
On Fri, Jun 18, 2010 at 10:29 AM, Benjamin Black wrote:
> In 0.6 your only option with those constraints is to iterate over the
> entire CF and deleting row by row. This requires you are eithe
What's the current timeframe on 0.7?
On Fri, Jun 18, 2010 at 1:45 PM, Benjamin Black wrote:
> Perfect, ship it.
>
> On Fri, Jun 18, 2010 at 10:37 AM, Stu Hood wrote:
> > See https://issues.apache.org/jira/browse/CASSANDRA-981
> >
> > -Original Message-
> > From: "Benjamin Black"
> > Se
*Hopefully* fixed. I was never able to duplicate the problem on my
workstation, but I had a pretty good idea what was causing the
problem. Julie, if you're in a position to apply and test the fix, it
would help help us make sure we've got this one nailed down.
Gary.
On Thu, Jun 17, 2010 at 00:3
In 0.6.x the iterating approach works ... but you need to flush and compact
(after GCGraceSeconds) in order to NOT see the keys in the CF.
Will the behavior of the truncate method in 0.7 require flush/compact as well?
Or will it be immediate?
-phil
On Jun 18, 2010, at 1:29 PM, Benjamin Black w
it will be immediate.
But it will fail if not all hosts in the cluster are up, this is the
tradeoff. We regard the truncate operation an admin api so I think it's a
fair tradeoff.
On Fri, Jun 18, 2010 at 11:50 PM, Phil Stanhope wrote:
> In 0.6.x the iterating approach works ... but you need to f
I am happy with this restriction on truncate operation for 0.7. Thanks for the
quick response.
-phil
On Jun 18, 2010, at 4:57 PM, Ran Tavory wrote:
> it will be immediate.
> But it will fail if not all hosts in the cluster are up, this is the
> tradeoff. We regard the truncate operation an ad
On Fri, 2010-06-18 at 11:00 -0700, Paul Brown wrote:
> At the risk of asking about religion (but with no interest in hearing
> about it), why Avro instead of something like plain-old-JSON over
> HTTP?
At the risk of having this thread veer off on a very long tangent...
In a nutshell, we need a wa
As mentioned in the #cassandra IRC channel - there's going to be a dinner in
Austin on July 15th for people interested in Cassandra.
For those interested: http://cassandradinneraustin.eventbrite.com/
(Sorry if this doesn't apply to everyone, but everyone is welcome :)
We are using MapReduce to periodical verify and rebuild our secondary
indexes along with counting total records. We started to noticed double
counting of unique keys on single machine standalone tests. We were finally
able to reproduce the problem using
the apache-cassandra-0.6.2-src/contrib/word_
"blow all the data away" ... how do you do that? What is the timestamp
precision that you are using when creating key/col or key/supercol/col items?
I have seen a fail to write a key when the timestamp is identical to the
previous timestamp of a deleted key/col. While I didn't examine the source
I thought the same thing, but using the supplied contrib example I just
delete the /var/lib/data dirs and commit log.
-Corey
On Fri, Jun 18, 2010 at 3:11 PM, Phil Stanhope wrote:
> "blow all the data away" ... how do you do that? What is the timestamp
> precision that you are using when creat
On Jun 18, 2010, at 2:12 PM, Eric Evans wrote:
> On Fri, 2010-06-18 at 11:00 -0700, Paul Brown wrote:
>> At the risk of asking about religion (but with no interest in hearing
>> about it), why Avro instead of something like plain-old-JSON over
>> HTTP?
> At the risk of having this thread veer off
OK...I just verified on a clean EC2 small single instance box using
apache-cassandra-0.6.2-src.
I'm pertty sure the Cassandra MapReduce functionality is broken.
If your MapReduce jobs are idempotent then you are OK, but if you are doing
things like word count (as in the supplied example) or key c
i'll jump in ... why AVRO over Thrift. can you guys point me at a
comparison? (i know next to nothing about both of them)
On 06/18/2010 03:41 PM, Paul Brown wrote:
On Jun 18, 2010, at 2:12 PM, Eric Evans wrote:
On Fri, 2010-06-18 at 11:00 -0700, Paul Brown wrote:
At the risk of a
On Fri, Jun 18, 2010 at 2:12 PM, Eric Evans wrote:
> On Fri, 2010-06-18 at 11:00 -0700, Paul Brown wrote:
>> At the risk of asking about religion (but with no interest in hearing
>> about it), why Avro instead of something like plain-old-JSON over
>> HTTP?
>
> At the risk of having this thread vee
On Fri, Jun 18, 2010 at 6:23 PM, Tatu Saloranta wrote:
> Not that I wanted to criticize choices, but do they actually allow use
> of JSON as encoding?
> Avro does use JSON for specifying schemas, but I wasn't aware of being
> able to use it for encoding data.
> Likewise with Thrift.
>
Yes, each
On Fri, Jun 18, 2010 at 4:57 PM, Miguel Verde wrote:
> On Fri, Jun 18, 2010 at 6:23 PM, Tatu Saloranta
> wrote:
>>
>> Not that I wanted to criticize choices, but do they actually allow use
>> of JSON as encoding?
>> Avro does use JSON for specifying schemas, but I wasn't aware of being
>> able to
I tried the following :
- always one cassandra node on one EC2 m.large instance. two other m.large
instance, I run 4 stress.py (50 thread each, 2 stress.py on each instance)
- RAID0 EBS for data and ephemeral EBS (/dev/sda1 partition) for commit log.
- -Xmx4G
and I did not see any improvements (Ca
On Jun 18, 2010, at 6:39 PM, Olivier Mallassi wrote:
> and I did not see any improvements (Cassandra stays around 7000 W/sec).
It's a brave new world where N+1 scaling with 7,000 writes per second per node
is considered suboptimal performance.
--Joe
Howdy!
So, last week I finally got around to playing with Cassandra. After a
while I understood the basics. To test this assumption I started
working on my own Client implementation since "Learning-by-doing" is
what I do and existing Ruby Clients (which are awesome) already
abstracted too much for
> @Chris, Did you get any bench you could share with us?
We're still working on it. It's a lower priority task so it will take a
while to finish. So far we've run on all the AWS data centers in the US
and used several different setups. We also did a test on Rackspace with
one setup and some whi
set log level to TRACE and see if the OutboundTcpConnection is going
bad. that would explain the message never arriving.
On Fri, Jun 18, 2010 at 10:39 AM, AJ Slater wrote:
> To summarize:
>
> If a request for a column comes in *after a period of several hours
> with no requests*, then the node s
My guess? 8-10 weeks.
On Fri, Jun 18, 2010 at 1:31 PM, James Golick wrote:
> What's the current timeframe on 0.7?
--
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com
Fixed for 0.6.3: https://issues.apache.org/jira/browse/CASSANDRA-1042
On Fri, Jun 18, 2010 at 2:49 PM, Corey Hulen wrote:
>
> We are using MapReduce to periodical verify and rebuild our secondary
> indexes along with counting total records. We started to noticed double
> counting of unique keys
I can't think of any scenario where leaving Cassandra idle would
affect the results returned. I think something else is going on here.
On Fri, Jun 18, 2010 at 2:05 AM, Manfred Muench
wrote:
> Hi,
>
> I have noticed the following behaviour (bug?) which I don't completely
> understand:
> 1. start
Awesome...thanks.
I just downloaded the patch and applied it and verified it fixes our
problems.
what's the ETA on 0.6.3? (debating on weather to tolerate it or maintain
our own 0.6.2+patch).
-Corey
On Fri, Jun 18, 2010 at 8:21 PM, Jonathan Ellis wrote:
> Fixed for 0.6.3: https://issues.apac
Looks like the end of June.
On Fri, Jun 18, 2010 at 8:38 PM, Corey Hulen wrote:
> Awesome...thanks.
> I just downloaded the patch and applied it and verified it fixes our
> problems.
> what's the ETA on 0.6.3? (debating on weather to tolerate it or maintain
> our own 0.6.2+patch).
> -Corey
>
> O
Hi Maxim,
Lucandra doesn't support numeric queries quite yet. A workaround would
be to load your numbers and convert them to strings.
I'll eventually add support for this. Please feel free to help out if
you can :)
Jake
On Jun 17, 2010, at 1:16 PM, Maxim Kramarenko
wrote:
Hello!
44 matches
Mail list logo