> A few days after writing the data, we tried this on cassandra-cli
The default consistency level in the CLI is ONE, did you change it to LOCAL 
QUOURM ? 

(I'm assuming your example is for two reads from the same CF)

It looks like the first read was done at a lower CL, and the value returned is 
valid in the sense that one replica did not have any data. Behind the scenes 
Read Repair was active on the request and it repaired the one replica the first 
read was from. So the next time round the value was there. 

If you want strongly consistent behaviour using QUOURM or LOCAL QUOURM for both 
reads and writes. 

Cheers  

-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 4/12/2012, at 9:33 AM, Owen Davies <cassan...@obduk.com> wrote:

> We have written a large amount of data to Cassandra from another
> database. When writing the client was set to write local quorum.
> 
> A few days after writing the data, we tried this on cassandra-cli
> 
> get example['key'][123];
> Value was not found
> Elapsed time: 50 msec(s).
> 
> Then a bit later
> 
> get datapoints['key'][123];
> => (column=123, value=456, timestamp=1354095697384001)
> Elapsed time: 77 msec(s).
> 
> We assume this is to do with replication, with the first read causing
> a repair, but there dosen't seem to be any way of seeing what date is
> on which servers to validate.
> 
> I have not had a chance yet of trying the previous suggestion.
> 
> Owen
> 
> On 3 December 2012 20:18, aaron morton <aa...@thelastpickle.com> wrote:
>> When reading, sometimes the data is there,
>> sometimes it is not, which we think is a replication issue, even
>> though we have left it plenty of time after the writes.
>> 
>> Can you provide some more information on this ?
>> Are you talking about writes to one DC and reads from another ?
>> 
>> Cheers
>> 
>> -----------------
>> Aaron Morton
>> Freelance Cassandra Developer
>> New Zealand
>> 
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 4/12/2012, at 6:45 AM, Шамим <sre...@yandex.ru> wrote:
>> 
>> Yes, it's should be. dc1:3 means u have 3 copy of every piece of row, with
>> local quorum you always get a good consistency from 3 nodes.
>> First you have to calculate token for data center dc1 and add offset 100 to
>> token for the second data center which will resolve your problem. After
>> creating the keyspace you can run the nodetool command with ring KEYSPACE
>> NAME which should show u the load for every node as 33%
>> Hope it will help u
>> Shamim
>> 
>> Hi Shamim
>> 
>> I have read a bit about the Tokens. I understand how that could effect
>> the data distribution at first, but I don't understand if we have
>> specified Options: [dc1:3, dc2:3], surely after a while all the data
>> will be on every server?
>> 
>> Thanks,
>> 
>> Owen
>> 
>> On 3 December 2012 14:06, Шамим <sre...@yandex.ru> wrote:
>> 
>> Hello Owen,
>> Seems you did not configure token for all nodes correctly. See the section
>> Calculating Tokens for multiple data centers here
>> http://www.datastax.com/docs/0.8/install/cluster_init
>> 
>> Best regards
>> Shamim
>> ---
>> On Mon, Dec 3, 2012 at 4:42 PM, Owen Davies <cassan...@obduk.com> wrote:
>> 
>> We have a 2 data center test cassandra setup running, and are writing
>> to it using LOCAL_QUORUM. When reading, sometimes the data is there,
>> sometimes it is not, which we think is a replication issue, even
>> though we have left it plenty of time after the writes.
>> 
>> We have the following setup:
>> 
>> cassandra -v: 1.1.6
>> 
>> cassandra.yaml
>> -----------------------
>> 
>> cluster_name: something
>> 
>> endpoint_snitch: PropertyFileSnitch
>> 
>> cassandra-topology.properties
>> --------------------------------------------
>> 192.168.1.1=dc1:rack1
>> 192.168.1.2=dc1:rack1
>> 192.168.1.3=dc1:rack1
>> 
>> 192.168.2.1=dc2:rack1
>> 192.168.2.2=dc2:rack1
>> 192.168.2.3=dc3:rack1
>> 
>> default=nodc:norack
>> 
>> cassandra-cli
>> --------------------
>> Keyspace: example:
>> Replication Strategy: org.apache.cassandra.locator.NetworkTopologyStrategy
>> Durable Writes: true
>> Options: [dc1:3, dc2:3]
>> 
>> nodetool ring
>> -------------------
>> 
>> Address DC Rack Status State Load
>> Effective-Ownership Token
>> 
>> 159447687142037741049740936276011715300
>> 192.168.1.2 dc1 rack1 Up Normal 111.17 GB
>> 100.00% 67165620003619490909052924699950283577
>> 192.168.1.1 dc1 rack1 Up Normal 204.57 GB
>> 100.00% 71045951808949151217931264995073558408
>> 192.168.2.1 dc2 rack1 Up Normal 209.92 GB
>> 100.00% 107165019770579893816561717940612111506
>> 192.168.1.3 dc1 rack1 Up Normal 209.92 GB
>> 100.00% 114165363957966360026729000065495595953
>> 192.168.2.3 dc2 rack1 Up Normal 198.22 GB
>> 100.00% 147717787092318068320268200174271353451
>> 192.168.2.2 dc2 rack1 Up Normal 179.31 GB
>> 100.00% 159447687142037741049740936276011715300
>> 
>> Does anyone have any ideas why every server does not have the same
>> amount of data on?
>> 
>> Thanks,
>> 
>> Owen
>> 
>> 
>> 
>> --
>> 
>> 

Reply via email to