Re: Replication factor, LOCAL_QUORUM write consistency and materialized views

2024-05-17 Thread Gábor Auth
Hi, On Fri, May 17, 2024 at 6:18 PM Jon Haddad wrote: > I strongly suggest you don't use materialized views at all. There are > edge cases that in my opinion make them unsuitable for production, both in > terms of cluster stability as well as data integrity. > Oh, there is already an open and

Re: Replication factor, LOCAL_QUORUM write consistency and materialized views

2024-05-17 Thread Gábor Auth
Hi, On Fri, May 17, 2024 at 6:18 PM Jon Haddad wrote: > I strongly suggest you don't use materialized views at all. There are > edge cases that in my opinion make them unsuitable for production, both in > terms of cluster stability as well as data integrity. > I totally agree with you about it

Re: Replication factor, LOCAL_QUORUM write consistency and materialized views

2024-05-17 Thread Jon Haddad
I strongly suggest you don't use materialized views at all. There are edge cases that in my opinion make them unsuitable for production, both in terms of cluster stability as well as data integrity. Jon On Fri, May 17, 2024 at 8:58 AM Gábor Auth wrote: > Hi, > > I know, I know, the materialize

Replication factor, LOCAL_QUORUM write consistency and materialized views

2024-05-17 Thread Gábor Auth
Hi, I know, I know, the materialized view is experimental... :) So, I ran into a strange error. Among others, I have a very small 4-nodes cluster, with very minimal data (~100 MB at all), the keyspace's replication factor is 3, everything is works fine... except: if I restart a node, I get a lot

Recommended read/write consistency level for counters

2014-09-09 Thread Eugene Voytitsky
What is recommended read/write consistency level (CL) for counters? Yes I know that write_CL + read_CL > RF is recommended. But, I got strange results when run my junit tests with different CLs against 3 nodes cluster. I checked 9 combinations: (write=ONE,QUORUM,ALL) x (read=ONE,QUORUM,

Setting the read/write consistency globaly in the CQL3 datastax java driver

2014-05-15 Thread Sebastian Schmidt
Hi, I'm using the CQL3 Datastax Cassandra Java client. I want to use a global read and write consistency for my queries. I know that I can set the consistencyLevel for every single prepared statement. But I want to do that just once per cluster or once per session. Is that possible? Kind Re

Re: Read/Write consistency issue

2014-01-10 Thread Robert Wille
, 2014 at 4:59 PM To: Subject: Re: Read/Write consistency issue As was pointed out earlier, Consistency.ALL is still subject to the possibility of clock drift between nodes, and there is also the problem of using the exact same timestamp, which is increasingly likely to happen the faster you update

Re: Read/Write consistency issue

2014-01-10 Thread Manoj Khangaonkar
old is the value that was read from the column. val is the value that needs to be added to it. new is (old + val) that is written back to the column. regards On Fri, Jan 10, 2014 at 4:36 PM, Andrey Ilinykh wrote: > For single thread, consistency ALL it should work. I believe you do > somethin

Re: Read/Write consistency issue

2014-01-10 Thread Andrey Ilinykh
For single thread, consistency ALL it should work. I believe you do something different. What are these three numbers exactly? old=60616 val =19 new =60635 On Fri, Jan 10, 2014 at 1:50 PM, Manoj Khangaonkar wrote: > Hi > > Using Cassandra 2.0.0. > 3 node cluster > Replication 2. > Using consiste

Re: Read/Write consistency issue

2014-01-10 Thread Steven A Robenalt
>>> On Fri, Jan 10, 2014 at 2:35 PM, Robert Wille wrote: >>> >>>> Actually, locking won’t fix the problem. He’s getting the problem on a >>>> single thread. >>>> >>>> I’m pretty sure that if updates can occur within the same millisecond

Re: Read/Write consistency issue

2014-01-10 Thread Tupshin Harper
> >>> On Fri, Jan 10, 2014 at 2:35 PM, Robert Wille wrote: >>> >>>> Actually, locking won’t fix the problem. He’s getting the problem on a >>>> single thread. >>>> >>>> I’m pretty sure that if updates can occur within the same millisec

Re: Read/Write consistency issue

2014-01-10 Thread Steven A Robenalt
its epidermal layer”.. > > > > *From:* Tupshin Harper [mailto:tups...@tupshin.com] > *Sent:* Friday, January 10, 2014 5:13 PM > > *To:* user@cassandra.apache.org > *Subject:* Re: Read/Write consistency issue > > > > It is bad because of the risk of concurrent m

Re: Read/Write consistency issue

2014-01-10 Thread Manoj Khangaonkar
is clock skew), there is literally nothing you can do to >>> make this pattern work. >>> >>> Robert >>> >>> From: Todd Carrico >>> Reply-To: >>> Date: Friday, January 10, 2014 at 3:28 PM >>> To: "user@cassandra.apache.org&quo

RE: Read/Write consistency issue

2014-01-10 Thread Todd Carrico
m: Tupshin Harper [mailto:tups...@tupshin.com] Sent: Friday, January 10, 2014 5:13 PM To: user@cassandra.apache.org Subject: Re: Read/Write consistency issue It is bad because of the risk of concurrent modifications. If you don't have some kind of global lock on the document/row, then 2 readers mig

RE: Read/Write consistency issue

2014-01-10 Thread Todd Carrico
, January 10, 2014 4:52 PM To: user@cassandra.apache.org Subject: Re: Read/Write consistency issue Interested in knowing more on why read-before-write is an anti-pattern. In the next month or so, I intend to use Cassandra as a doc store. One very common operation will be to read the document, make a

Re: Read/Write consistency issue

2014-01-10 Thread Tupshin Harper
Robert > > From: Steven A Robenalt > Reply-To: > Date: Friday, January 10, 2014 at 3:41 PM > > To: > Subject: Re: Read/Write consistency issue > > My understanding is that it's generally a Cassandra anti-pattern to do > read-before-write in any case, not jus

Re: Read/Write consistency issue

2014-01-10 Thread Tupshin Harper
re is literally nothing you can do to >> make this pattern work. >> >> Robert >> >> From: Todd Carrico >> Reply-To: >> Date: Friday, January 10, 2014 at 3:28 PM >> To: "user@cassandra.apache.org" >> Subject: RE: Read/Write consistency is

Re: Read/Write consistency issue

2014-01-10 Thread Robert Wille
rapid repeated writing is not an issue. Why would this be bad? Robert From: Steven A Robenalt Reply-To: Date: Friday, January 10, 2014 at 3:41 PM To: Subject: Re: Read/Write consistency issue My understanding is that it's generally a Cassandra anti-pattern to do read-before-write i

Re: Read/Write consistency issue

2014-01-10 Thread Steven A Robenalt
ake this pattern work. > > Robert > > From: Todd Carrico > Reply-To: > Date: Friday, January 10, 2014 at 3:28 PM > To: "user@cassandra.apache.org" > Subject: RE: Read/Write consistency issue > > That, or roll your own locking. Means multiple updates, but

RE: Read/Write consistency issue

2014-01-10 Thread Todd Carrico
Is it possible to pin to a node, instead of letting the client find the next node (round robin)? Sorry, a C* noob here... tc From: Robert Wille [mailto:rwi...@fold3.com] Sent: Friday, January 10, 2014 4:35 PM To: user@cassandra.apache.org Subject: Re: Read/Write consistency issue Actually

Re: Read/Write consistency issue

2014-01-10 Thread Robert Wille
: Date: Friday, January 10, 2014 at 3:28 PM To: "user@cassandra.apache.org" Subject: RE: Read/Write consistency issue That, or roll your own locking. Means multiple updates, but it works reliably. tc From: Robert Wille [mailto:rwi...@fold3.com] Sent: Friday, January 10, 2014

RE: Read/Write consistency issue

2014-01-10 Thread Todd Carrico
That, or roll your own locking. Means multiple updates, but it works reliably. tc From: Robert Wille [mailto:rwi...@fold3.com] Sent: Friday, January 10, 2014 4:25 PM To: user@cassandra.apache.org Subject: Re: Read/Write consistency issue Cassandra is a last-write wins kind of a deal. The last

Re: Read/Write consistency issue

2014-01-10 Thread Robert Wille
at 2:50 PM To: Subject: Read/Write consistency issue Hi Using Cassandra 2.0.0. 3 node cluster Replication 2. Using consistency ALL for both read and writes. I have a single thread that reads a value, updates it and writes it back to the table. The column type is big int. Updating counts for a

Read/Write consistency issue

2014-01-10 Thread Manoj Khangaonkar
Hi Using Cassandra 2.0.0. 3 node cluster Replication 2. Using consistency ALL for both read and writes. I have a single thread that reads a value, updates it and writes it back to the table. The column type is big int. Updating counts for a timestamp. With single thread and consistency ALL , I e

Re: How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9

2013-05-06 Thread aaron morton
Regards, > Praveen > > > > > > From: aaron morton > mailto:aa...@thelastpickle.com>> > Reply-To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" > mailto:user@cassandra.apache.org>> > Date: Tuesday, April 30, 2013 1:47 AM > To: &q

Re: How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9

2013-05-05 Thread praveen.akunuru
t;" mailto:user@cassandra.apache.org>> Date: Tuesday, April 30, 2013 1:47 AM To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" mailto:user@cassandra.apache.org>> Subject: Re: How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9 One o

Re: How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9

2013-04-29 Thread Robert Coli
On Mon, Apr 29, 2013 at 1:17 PM, aaron morton wrote: > Bulk Loader does not use CL, it's more like a repair / bootstrap. > If you have to skip a node then use repair. The bulk loader ("sstableloader") can ignore replica nodes via -i option : ./src/java/org/apache/cassandra/tools/BulkLoader.java

Re: How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9

2013-04-29 Thread aaron morton
we can enforce Write Consistency level when > using SSTABLELOADER? Bulk Loader does not use CL, it's more like a repair / bootstrap. If you have to skip a node then use repair. Cheers - Aaron Morton Freelance Cassandra Consultant New Zealand @aaronmorton http://www.thelas

How to use Write Consistency 'ANY' with SSTABLELOADER - DSE Cassandra 1.1.9

2013-04-29 Thread praveen.akunuru
R as well. We tried specifying the consistency level 'ANY' at Keyspace level. However, this is not being used by the SSTABLELOADER. It is still looking for all the nodes to be available. 1. Please can anyone suggest how we can enforce Write Consistency level when using SSTABLELOADER?

Re: What if write consistency level cannot me met ?

2011-02-15 Thread Aaron Morton
the write has reached  at least one node, > it will eventually reach all the other nodes as well. So it won't > rollback. > > > On Tue, Feb 15, 2011 at 7:38 PM, A J <s5a...@gmail.com> wrote: >> Say I set write consistency level to ALL and all but one node are down. W

Re: What if write consistency level cannot me met ?

2011-02-15 Thread Matthew Dennis
nodes as well. So it won't > > rollback. > > > > > > On Tue, Feb 15, 2011 at 7:38 PM, A J wrote: > >> Say I set write consistency level to ALL and all but one node are down. > What > >> happens to writes ? Does it rollback from the live node before returning > >> failure to client ? > >> Thanks. >

Re: What if write consistency level cannot me met ?

2011-02-15 Thread Aaron Morton
hed at least one node, > it will eventually reach all the other nodes as well. So it won't > rollback. > > > On Tue, Feb 15, 2011 at 7:38 PM, A J wrote: >> Say I set write consistency level to ALL and all but one node are down. What >> happens to writes ? Does it

Re: What if write consistency level cannot me met ?

2011-02-15 Thread Thibaut Britz
Your write will fail. But if the write has reached at least one node, it will eventually reach all the other nodes as well. So it won't rollback. On Tue, Feb 15, 2011 at 7:38 PM, A J wrote: > Say I set write consistency level to ALL and all but one node are down. What > happen

What if write consistency level cannot me met ?

2011-02-15 Thread A J
Say I set write consistency level to ALL and all but one node are down. What happens to writes ? Does it rollback from the live node before returning failure to client ? Thanks.

RE: per-connection "read-after-my-write" consistency

2011-02-12 Thread Dan Hendry
mic...@gmail.com] Sent: February-12-11 17:37 To: user@cassandra.apache.org Subject: Re: per-connection "read-after-my-write" consistency Hi, I'm using .NET and I wrote my own client library (over Thrift) so I'm absolutely sure that both operations are performed using the

RE: per-connection "read-after-my-write" consistency

2011-02-12 Thread Dan Hendry
[mailto:augustyn.mic...@gmail.com] Sent: February-12-11 4:13 To: user@cassandra.apache.org Subject: per-connection "read-after-my-write" consistency Hi, I'm running 2 nodes with RF=2 (not optimal, I know), Cassandra 0.7.1. During one connection, I write (CL.ONE) a row and subsequent

Re: per-connection "read-after-my-write" consistency

2011-02-12 Thread Michal Augustýn
> > > Dan > > > > *From:* Michal Augustýn [mailto:augustyn.mic...@gmail.com] > *Sent:* February-12-11 4:13 > *To:* user@cassandra.apache.org > *Subject:* per-connection "read-after-my-write" consistency > > > > Hi, > > > > I'm running 2 nodes

per-connection "read-after-my-write" consistency

2011-02-12 Thread Michal Augustýn
true for most cases, but circa 1 of 1000 attempts doesn't work as expected - I get no row :( Where is the problem please? Should I use another CL for read and/or write? I would like just to achieve "per connection read-after-my-write consistency". Thank you very much! Augi

Re: about the write consistency

2011-01-12 Thread Brandon Williams
2011/1/12 raoyixuan (Shandy) > if I have 20 nodes, and replica factor is 3, whether all the node have > the replica finally or just have 3 replica? > 3. -Brandon

about the write consistency

2011-01-12 Thread raoyixuan (Shandy)
if I have 20 nodes, and replica factor is 3, whether all the node have the replica finally or just have 3 replica? 华为技术有限公司 Huawei Technologies Co., Ltd.[Company_logo] Phone: 28358610 Mobile: 13425182943 Email: raoyix...@huawei.com 地址:深圳市龙岗区坂田华为基地 邮编:518129 Huawei

Re: Write consistency

2010-04-08 Thread Avinash Lakshman
Retry is the best option. Because the read repair will fix it on a subsequent read and it will actually fix it with a value that was actually deemed a failed write to the client. Avinash On Thu, Apr 8, 2010 at 9:47 AM, David Strauss wrote: > A read repair will fix it immediately after the first

Re: Write consistency

2010-04-08 Thread David Strauss
A read repair will fix it immediately after the first read of the row. On 2010-04-08 16:36, Mark Greene wrote: > So unless you re-try the write, the previous stale write stays on the > other two nodes? Would a read repair fix this eventually? > > On Thu, Apr 8, 2010 at 11:36 AM, Avinash Lakshman

Re: Write consistency

2010-04-08 Thread Benjamin Black
His arguments consistently (hah!) boil down to this: if you misconfigure things for your intended application, you get undesirable behavior. For example, the correct approach to the situation cited is to use quorum reads and writes. W=3/R=1/N=3 might be appropriate for situations in which you wan

Re: Write consistency

2010-04-08 Thread Benjamin Black
Yes. Or you would retry the write. Either way, the system achieves consistency eventually, hence the name. On Thu, Apr 8, 2010 at 9:36 AM, Mark Greene wrote: > So unless you re-try the write, the previous stale write stays on the other > two nodes? Would a read repair fix this eventually? >

Re: Write consistency

2010-04-08 Thread Mark Greene
So unless you re-try the write, the previous stale write stays on the other two nodes? Would a read repair fix this eventually? On Thu, Apr 8, 2010 at 11:36 AM, Avinash Lakshman < avinash.laksh...@gmail.com> wrote: > What your describing is a distributed transaction? Generally strong > consistenc

Re: Write consistency

2010-04-08 Thread Benjamin Black
On Thu, Apr 8, 2010 at 12:55 AM, Paul Prescod wrote: > > ¹ http://jsensarma.com/blog/2009/11/dynamo-part-i-a-followup-and-re-rebuttals/ > Pay no attention to this disingenuous troll. b

Re: Write consistency

2010-04-08 Thread Avinash Lakshman
What your describing is a distributed transaction? Generally strong consistency is always associated with doing transactional writes where you never see the results of a failed write on a subsequent read no matter what happens. Cassandra has no notion of rollback. That is why no combination will gi

Re: Write consistency

2010-04-08 Thread Jeremy Dunck
On Thu, Apr 8, 2010 at 7:16 AM, Gary Dusbabek wrote: > On Thu, Apr 8, 2010 at 02:55, Paul Prescod wrote: >> In this¹ debate, there seemed to be consensus on the following fact: >> >> "In Cassandra, say you use N=3, W=3 & R=1. Let’s say you managed to >> only write to replicas A & B, but not C. In

Re: Write consistency

2010-04-08 Thread Gary Dusbabek
On Thu, Apr 8, 2010 at 02:55, Paul Prescod wrote: > In this¹ debate, there seemed to be consensus on the following fact: > > "In Cassandra, say you use N=3, W=3 & R=1. Let’s say you managed to > only write to replicas A & B, but not C. In this case Cassandra will > return an error to the applicati

Write consistency

2010-04-08 Thread Paul Prescod
In this¹ debate, there seemed to be consensus on the following fact: "In Cassandra, say you use N=3, W=3 & R=1. Let’s say you managed to only write to replicas A & B, but not C. In this case Cassandra will return an error to the application saying the write failed- which is acceptable given than W