Re: New Chain for : Does Cassandra use vector clocks

2011-02-24 Thread Dave Revell
>Time stamps are not used for conflict resolution - unless is is part of the
application logic!!!

This is false. In fact, the main reason Cassandra keeps timestamps is to do
conflict resolution. If there is a conflict between two replicas, when doing
a read or a repair, then the highest timestamp always wins.

Example: say your replication factor is 5. So if you read at CL ALL, you
will ask 5 replicas for their value. If the value from only one of these
replicas has a timestamp that is newer than all the rest, this is the value
that will be retruned to the client. There is no "voting" scheme where the
most common value wins, the conflict resolution is based ONLY on the most
recent timestamp.

(irrelevant aside: in the above example, read repair would occur at the end,
after the different values were detected by the coordinating server)

Clients are free to use the timestamps for their own purposes, but clients
must be careful to choose timestamps that make Cassandra do the right thing
during conflict resolution.

Best,
Dave

On Thu, Feb 24, 2011 at 8:34 AM, Anthony John  wrote:

> >>Time stamps are not used for conflict resolution - unless is is part of
>> the application logic!!!
>>
>
> >>What is you definition of conflict resolution ? Because if you update
> twice the same column (which
> >>I'll call a conflict), then the timestamps are used to decide which
> update wins (which I'll call a resolution).
>
> I understand what you are saying, and yes semantics is very important here.
> And yes we are responding to the immediate questions without covering all
> questions in the thread.
>
> The point being made here is that the timestamp of the column is not used
> by Cassandra to figure out what data to return.
>
> E.g. - Quorum is 2 nodes - and RF of 3 over N1/2/3
> A Quorum  Write comes and add/updates the time stamp (TS2) of a particular
> data element. It succeeds on N1 - fails on N2/3. So the write is returned as
> failed - right ?
> Now Quorum read comes in for exactly the same piece of data that the write
> failed for.
> So N1 has TS2 but both N2/3 have the old TS (say TS1)
> And the read succeeds - Will it return TS1 or TS2.
>
> I submit it will return TS1 - the old TS.
>
> Are we on the same page with this interpretation ?
>
> Regards,
>
> -JA
>
> On Thu, Feb 24, 2011 at 10:12 AM, Sylvain Lebresne 
> wrote:
>
>> On Thu, Feb 24, 2011 at 4:52 PM, Anthony John wrote:
>>
>>> Sylvan,
>>>
>>> Time stamps are not used for conflict resolution - unless is is part of
>>> the application logic!!!
>>>
>>
>> What is you definition of conflict resolution ? Because if you update
>> twice the same column (which
>> I'll call a conflict), then the timestamps are used to decide which update
>> wins (which I'll call a resolution).
>>
>>
>>> You can have "lost updates" w/Cassandra. You need to to use 3rd products
>>> - cages for e.g. - to get ACID type consistency.
>>>
>>
>> Then again, you'll have to define what you are calling "lost updates".
>> Provided you use a reasonable consistency level, Cassandra provides fairly
>> strong durability guarantee, so for some definition you don't "lose
>> updates".
>>
>> That being said, I never pretended that Cassandra provided any ACID
>> guarantee. ACID relates to transaction, which Cassandra doesn't support. If
>> we're talking about the guarantees of transaction, then by all means,
>> cassandra won't provide it. And yes you can use cages or the like to get
>> transaction. But that was not the point of the thread, was it ? The thread
>> is about vector clocks, and that has nothing to do with transaction (vector
>> clocks certainly don't give you transactions).
>>
>> Sorry if I wasn't clear in my mail, but I was only responding to why so
>> far I don't think vector clocks would really provide much for Cassandra.
>>
>> --
>> Sylvain
>>
>>
>>> -JA
>>>
>>>
>>> On Thu, Feb 24, 2011 at 7:41 AM, Sylvain Lebresne 
>>> wrote:
>>>
 On Thu, Feb 24, 2011 at 3:22 AM, Anthony John wrote:

> Apologies : For some reason my response on the original mail keeps
> bouncing back, thus this new one!
> > From the other hand, the same article says:
> > "For conditional writes to work, the condition must be evaluated at
> all update
> > sites before the write can be allowed to succeed."
> >
> > This means, that when doing such an update CL=ALL must be used
>
> Sorry, but I am confused by that entire thread!
>
> Questions:-
> 1. Does Cassandra implement any kind of data locking - at any
> granularity whether it be row/colF/Col ?
>

 No locking, no.


> 2. If the answer to 1 above is NO! - how does CL ALL prevent conflicts.
> Concurrent updates on exactly the same piece of data on different nodes 
> can
> still mess each other up, right ?
>

 Not sure why you are taking CL.ALL specifically. But in any CL, updating
 the same piece of data means the same column value. In that case, the
 

Re: How does Cassandra handle failure during synchronous writes

2011-02-23 Thread Dave Revell
Ritesh,

You have seen the problem. Clients may read the newly written value even
though the client performing the write saw it as a failure. When the client
reads, it will use the correct number of replicas for the chosen CL, then
return the newest value seen at any replica. This "newest value" could be
the result of a failed write.

Read repair will probably occur at that point (depending on your config),
which would cause the newest value to propagate to more replicas.

R+W>N guarantees serial order of operations: any read at CL=R that occurs
after a write at CL=W will observe the write. I don't think this property is
relevant to your current question, though.

Cassandra has no mechanism to "roll back" the partial write, other than to
simply write again. This may also fail.

Best,
Dave


On Wed, Feb 23, 2011 at 10:12 AM,  wrote:

> Hi Dave,
> Thanks for your input. In the steps you mention, what happens when client
> tries to read the value at step 6? Is it possible that the client may see
> the new value? My understanding was if R + W > N, then client will not see
> the new value as Quorum nodes will not agree on the new value. If that is
> the case, then its alright to return failure to the client. However, if not,
> then it is difficult to program as after every failure, you as an client are
> not sure if failure is a pseudo failure with some side effects or real
> failure.
>
> Thanks,
> Ritesh
>
> 
> Ritesh,
>
> There is no commit protocol. Writes may be persisted on some replicas even
> though the quorum fails. Here's a sequence of events that shows the
> "problem:"
>
> 1. Some replica R fails, but recently, so its failure has not yet been
> detected
> 2. A client writes with consistency > 1
> 3. The write goes to all replicas, all replicas except R persist the write
> to disk
> 4. Replica R never responds
> 5. Failure is returned to the client, but the new value is still in the
> cluster, on all replicas except R.
>
> Something very similar could happen for CL QUORUM.
>
> This is a conscious design decision because a commit protocol would
> constitute tight coupling between nodes, which goes against the Cassandra
> philosophy. But unfortunately you do have to write your app with this case
> in mind.
>
> Best,
> Dave
>
> On Tue, Feb 22, 2011 at 8:22 PM, tijoriwala.ritesh <
> tijoriwala.rit...@gmail.com> wrote:
>
> >
> > Hi,
> > I wanted to get details on how does cassandra do synchronous writes to W
> > replicas (out of N)? Does it do a 2PC? If not, how does it deal with
> > failures of of nodes before it gets to write to W replicas? If the
> > orchestrating node cannot write to W nodes successfully, I guess it will
> > fail the write operation but what happens to the completed writes on X (W
> > >
> > X) nodes?
> >
> > Thanks,
> > Ritesh
> > --
> > View this message in context:
> >
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/How-does-Cassandra-handle-failure-during-synchronous-writes-tp6055152p6055152.html
> > Sent from the cassandra-u...@incubator.apache.org mailing list archive
> at
> > Nabble.com.
> >
>
> 
> Quoted from:
>
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/How-does-Cassandra-handle-failure-during-synchronous-writes-tp6055152p6055408.html
>


Re: How does Cassandra handle failure during synchronous writes

2011-02-22 Thread Dave Revell
Ritesh,

There is no commit protocol. Writes may be persisted on some replicas even
though the quorum fails. Here's a sequence of events that shows the
"problem:"

1. Some replica R fails, but recently, so its failure has not yet been
detected
2. A client writes with consistency > 1
3. The write goes to all replicas, all replicas except R persist the write
to disk
4. Replica R never responds
5. Failure is returned to the client, but the new value is still in the
cluster, on all replicas except R.

Something very similar could happen for CL QUORUM.

This is a conscious design decision because a commit protocol would
constitute tight coupling between nodes, which goes against the Cassandra
philosophy. But unfortunately you do have to write your app with this case
in mind.

Best,
Dave

On Tue, Feb 22, 2011 at 8:22 PM, tijoriwala.ritesh <
tijoriwala.rit...@gmail.com> wrote:

>
> Hi,
> I wanted to get details on how does cassandra do synchronous writes to W
> replicas (out of N)? Does it do a 2PC? If not, how does it deal with
> failures of of nodes before it gets to write to W replicas? If the
> orchestrating node cannot write to W nodes successfully, I guess it will
> fail the write operation but what happens to the completed writes on X (W >
> X) nodes?
>
> Thanks,
> Ritesh
> --
> View this message in context:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/How-does-Cassandra-handle-failure-during-synchronous-writes-tp6055152p6055152.html
> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
> Nabble.com.
>


Re: Patterns for writing enterprise applications on cassandra

2011-02-16 Thread Dave Revell
Re Anthony's statement:

> So it can be done and frameworks like CAGES are showing a way forward. At
> the heart of it, there will need to be a Two-Phase commit type protocol
> coordinator that sits in front of Cassandra. Of which - one can be sure -
there
> will be many implementations / best practices in the coming months.

I disagree. I think anyone who wants transactions should pick a database
that supports them. Bolting a transactional system on top could perhaps be
made to work at great cost if you always used CL ALL for every operation. I
personally don't think it's possible, but I can't actually prove it.

Consider how to enforce:
1) atomicity: you need some kind of undo/redo logging system with crash
recovery to handle partially-executed transactions. This is a lot of tricky
Cassandra-specific code. A locking system isn't good enough.
2) isolation: lock managers are f*&^ing hard, especially handling the
failure cases. Performant deadlock detection is difficult. Getting
sufficiently fine-grained locks would require Cassandra-specific code.

I'm trying to argue that these features belong inside the database, and not
bolted on top, so you should use a database that includes them.

Plainly: don't use Cassandra for applications that require
transactions. However, if you can express your app without the need of
transactions, that where Cassandra really shines.

+1 on Nate's recommendation to read the Helland paper.

Contentiously,
Dave

On Wed, Feb 16, 2011 at 2:20 PM, Nate McCall  wrote:

> I found the following paper (PDF) very helpful in shaping my thoughts
> about what it means to build systems without transactions.
>
> http://www.ics.uci.edu/~cs223/papers/cidr07p15.pdf
>
> "LIfe Beyond Distributed Transactions: an Apostate's Opinion" by Pat
> Helland
>
> On Wed, Feb 16, 2011 at 2:00 PM, tijoriwala.ritesh
>  wrote:
> >
> > Thanks a lot Anthony. That does help me think on possible options...
> > --
> > View this message in context:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Patterns-for-writing-enterprise-applications-on-cassandra-tp6030077p6033316.html
> > Sent from the cassandra-u...@incubator.apache.org mailing list archive
> at Nabble.com.
> >
>


Re: Patterns for writing enterprise applications on cassandra

2011-02-16 Thread Dave Revell
Ritesh,

There don't seem to be any common best practices to do this. I think the
reason is that by adding transaction semantics on top of Cassandra you're
throwing away the most important properties of Cassandra. The effects of a
transaction/locking layer:

- A centralized performance bottleneck that won't scale linearly
- Complex failure detection and recovery
- Reduced availability/partition tolerance (CAP: C prevents simultaneous A
and P)
- High latency for geographically remote clients
- Lower throughput due to enforced serial ordering of transactions

There are probably other reasons that didn't occur to me. Cassandra's great
at doing what it does, but it's certainly not a general purpose
transactional database for all use cases.

-Dave

On Wed, Feb 16, 2011 at 11:19 AM, tijoriwala.ritesh <
tijoriwala.rit...@gmail.com> wrote:

>
> Hi Gaurav,
> Thanks for the reply...I did look at the cages framework and I see that it
> provides some functionality for locking and atomic writes for multiple
> keys.
> My question was that do people rely on these kind of frameworks - if so, is
> cages the only one or are there others as well...and if not, what do they
> do
> to solve these kind of problems...
>
> Thanks,
> Ritesh
> --
> View this message in context:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Patterns-for-writing-enterprise-applications-on-cassandra-tp6030077p6033138.html
> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
> Nabble.com.
>


Re: Indexes and hard disk

2011-02-12 Thread Dave Revell
Indexes have another important advantage over multiple denormalized column
families. If you make the copies yourself, eventually the copies will
diverge from the base "true" column family due to routine occasional
failures. You'll probably want to find and fix these inconsistencies.

If you're using built-in indexes, you won't have this problem and you can
save some effort.

-Dave
On Feb 12, 2011 7:16 AM, "Bill de hÓra"  wrote:
> On Sat, 2011-02-12 at 14:03 +0100, Filip Nguyen wrote:
>
>
>> Why the secondary indexes are even present in Cassandra? I thought the
>> point is that development in Cassandra is query driven, that when you
>> want to search and fetch for example by birth date you should create
>> new ColumnFamilly...
>
>
>
> Yes and no. Systems like Cassandra are designed such that you should
> write the data out as you want to read it in (because writes are cheap).
> However most systems will want to access data via a few other criteria.
> For example a blogging system that supports tags will need to list your
> blog entries by date and by tag equally efficiently . As you say, you
> can spin up a new ColumnFamilly for that, but it's such a common need
> that Cassandra 0.7 supports it directly and saves developers having to
> manage indexes by hand (under the hood, a 0.7 index is a 'private' CF).
> This for me is one of the features that really sets Cassandra apart -
> scaling and indexing data at the same time is hard, and very few systems
> do both well.
>
> Bill


Re: Can serialized objects in columns serve as ersatz superCFs?

2011-02-08 Thread Dave Revell
Yes, this works well for me. I have no SCFs but many columns contain JSON.

Depending on your time/space/compatibility tradeoffs you can obviously pick
you own serialization method.

Best,
Dave
On Feb 8, 2011 4:33 AM, "buddhasystem"  wrote:
>
> Seeing that discussion here about indexes not supported in superCFs, and
less
> than clear future of superCFs altogether, I was thinking about getting a
> modicum of same functionality with serialized objects inside columns. This
> way the column key becomes sort of analog of supercolumn key, and I handle
> the dictionaries I receive in the client.
>
> Does this sound OK?
>
> --
> View this message in context:
http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Can-serialized-objects-in-columns-serve-as-ersatz-superCFs-tp6003775p6003775.html
> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
Nabble.com.


Re: TSocket timing out

2011-01-30 Thread Dave Revell
Under heavy load, this could be the result of the server not accept()ing
fast enough, causing the number of pending connections to exceed the listen
backlog size in the kernel.

I believe Cassandra uses the default of 50 backlogged connections.

This is one of the reasons why a persistent connection pool is a good idea.

-Dave
On Jan 30, 2011 1:14 AM, "aaron morton"  wrote:
> Am assuming these are client side side timeouts, you could increase the
client side timeout when the TSocket is created. Are you using a higher
level library or raw Thrift?.
>
> Alternatively you may be overloading the cluster. Are there are WARN
messages in the cluster about Dropped Messages ?
>
> Aaron
>
> On 30 Jan 2011, at 14:19, Courtney Robinson wrote:
>
>> It may also be an idea to check the node's memory usage. I encountered
this on a few occasions and I simply killed
>> any unneeded process that was eating away my node's memory. In each
instance it worked fine after there was about 300MB of free memory
>>
>> From: Patricio Echagüe
>> Sent: Sunday, January 30, 2011 12:46 AM
>> To: user@cassandra.apache.org
>> Subject: Re: TSocket timing out
>>
>> The recommendation is to wait few milliseconds and retry.
>>
>> For Example if you use Hector ( I don't think it is your case), Hector
will retry to different nodes in your cluster and the retry mechanisms is
tunable as well.
>>
>> On Sat, Jan 29, 2011 at 2:20 PM, buddhasystem  wrote:
>>
>> When I do a lot of inserts into my cluster (>10k at a time) I get
timeouts
>> from Thrift, the TScoket.py module.
>>
>> What do I do?
>>
>> Thanks,
>>
>> Maxim
>>
>> --
>> View this message in context:
http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/TSocket-timing-out-tp5973548p5973548.html
>> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
Nabble.com.
>>
>