Re: High disk I/O during reads

2013-03-23 Thread igor
You can try to disable readahead on cassandra data disk.

Jon Scarborough  написал(а):

>Checked tpstats, there are very few dropped messages.
>
>Checked histograms. Mostly nothing surprising. The vast majority of
>rows
>are small, and most reads only access one or two SSTables.
>
>What I did discover is that of our 5 nodes, one is performing well,
>with
>disk I/O in the ballprk that seems reasonable. The other 4 nodes are
>doing
>roughly 4x the disk i/O per second.  Interestingly, the node that is
>performing well also seems to be servicing about twice the number of
>reads
>that the other nodes are.
>
>I compared configuration between the node performing well to those that
>aren't, and so far haven't found any discrepancies.
>
>On Fri, Mar 22, 2013 at 10:43 AM, Wei Zhu  wrote:
>
>> According to your cfstats, read latency is over 100 ms which is
>really
>> really slow. I am seeing less than 3ms reads for my cluster which is
>on
>> SSD. Can you also check the nodetool cfhistorgram, it tells you more
>about
>> the number of SSTable involved and read/write latency. Somtimes
>average
>> doesn't tell you the whole storey.
>> Also check your nodetool tpstats, are there a lot dropped reads?
>>
>> -Wei
>> - Original Message -
>> From: "Jon Scarborough" 
>> To: user@cassandra.apache.org
>> Sent: Friday, March 22, 2013 9:42:34 AM
>> Subject: Re: High disk I/O during reads
>>
>> Key distribution across probably varies a lot from row to row in our
>case.
>> Most reads would probably only need to look at a few SSTables, a few
>might
>> need to look at more.
>>
>> I don't yet have a deep understanding of C* internals, but I would
>imagine
>> even the more expensive use cases would involve something like this:
>>
>> 1) Check the index for each SSTable to determine if part of the row
>is
>> there.
>> 2) Look at the endpoints of the slice to determine if the data in a
>> particular SSTable is relevant to the query.
>> 3) Read the chunks of those SSTables, working backwards from the end
>of
>> the slice until enough columns have been read to satisfy the limit
>clause
>> in the query.
>>
>> So I would have guessed that even the more expensive queries on wide
>rows
>> typically wouldn't need to read more than a few hundred KB from disk
>to do
>> all that. Seems like I'm missing something major.
>>
>> Here's the complete CF definition, including compression settings:
>>
>> CREATE COLUMNFAMILY conversation_text_message (
>> conversation_key bigint PRIMARY KEY
>> ) WITH
>> comment='' AND
>>
>comparator='CompositeType(org.apache.cassandra.db.marshal.DateType,org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.AsciiType,org.apache.cassandra.db.marshal.AsciiType)'
>> AND
>> read_repair_chance=0.10 AND
>> gc_grace_seconds=864000 AND
>> default_validation=text AND
>> min_compaction_threshold=4 AND
>> max_compaction_threshold=32 AND
>> replicate_on_write=True AND
>> compaction_strategy_class='SizeTieredCompactionStrategy' AND
>>
>>
>compression_parameters:sstable_compression='org.apache.cassandra.io.compress.SnappyCompressor';
>>
>> Much thanks for any additional ideas.
>>
>> -Jon
>>
>>
>>
>> On Fri, Mar 22, 2013 at 8:15 AM, Hiller, Dean < dean.hil...@nrel.gov
>>
>> wrote:
>>
>>
>> Did you mean to ask "are 'all' your keys spread across all SSTables"?
>I am
>> guessing at your intention.
>>
>> I mean I would very well hope my keys are spread across all sstables
>or
>> otherwise that sstable should not be there as he has no keys in it
>;).
>>
>> And I know we had HUGE disk size from the duplication in our sstables
>on
>> size-tiered compaction….we never ran a major compaction but after we
>> switched to LCS, we went from 300G to some 120G or something like
>that
>> which was nice. We only have 300 data point posts / second so not an
>> extreme write load on 6 nodes as well though these posts causes read
>to
>> check authorization and such of our system.
>>
>> Dean
>>
>> From: Kanwar Sangha < kan...@mavenir.com >>
>> Reply-To: " user@cassandra.apache.org user@cassandra.apache.org>" <
>> user@cassandra.apache.org >
>> Date: Friday, March 22, 2013 8:38 AM
>> To: " user@cassandra.apache.org "
><
>> user@cassandra.apache.org >
>> Subject: RE: High disk I/O during reads
>>
>>
>> Are your Keys spread across all SSTables ? That will cause every
>sstable
>> read which will increase the I/O.
>>
>> What compaction are you using ?
>>
>> From: zod...@fifth-aeon.net  [mailto:
>> zod...@fifth-aeon.net ] On Behalf Of Jon Scarborough
>>
>> Sent: 21 March 2013 23:00
>> To: user@cassandra.apache.org 
>>
>>
>> Subject: High disk I/O during reads
>>
>> Hello,
>>
>> We've had a 5-node C* cluster (version 1.1.0) running for several
>months.
>> Up until now we've mostly been writing data, but now we're starting
>to
>> service more read traffic. We're seeing far more disk I/O to service
>these
>> reads than I would have anticipated.
>>
>> The CF being queried consists of chat messages. Each row represents a
>> conversation betwe

Many to one type of replication.

2013-03-23 Thread Francois Richard
Hi,

We currently run our Cassandra deployment with
multiple independent clusters.  The clusters are totally self contain in
terms of redundancy and independent from each others.  We have a "sharding
"layer higher in our stack to dispatch the requests to the right
application stack and this stack connects to his associated Cassandra
cluster. All the cassandra clusters are identical in terms of hosted
keyspaces, column families, replication factor ...

At this point I am investigating ways to build a central cassandra cluster
that could contain all the data from all the other cassandra clusters and I
am wondering how to best do it.  The goal is to have a global view of our
data and to be able to do some massive crunching on it.

For sure we can build some ETL type of job that would figure out the data
that was updated, extract it, and load it to the central cassandra cluster.
 From this mailing list I found this Github project that is doing something
similar by looking at the commit logs:
https://github.com/carloscm/cassandra-commitlog-extract

But is there other options around using a custom replication strategy?  Any
other general suggestions ?

Thanks,

FR

-- 

_

*Francois Richard *


Cassandra - conflict resolution for column updates with identical timestamp

2013-03-23 Thread dong.yajun
Hello,

I would like to know which write wins in case of two updates with the
same client timestamp in Cassandra.

Initial data: KeyA: { col1:"val AA", col2:"val BB", col3:"val CC"}

Client 1 sends update: KeyA: { col1:"val C1", col2:"val B1"} on Sx

Client 2 sends update: KeyA: { col1:"val C2", col2:"val B2"} on Sy

Both updates have the same timestamp.

-- 
*Ric Dong*


Re: Lots of Deleted Rows Came back after upgrade 1.1.6 to 1.1.10

2013-03-23 Thread aaron morton
> Beside the joke, would hinted handoff really have any role in this issue?
I could imagine a  scenario where a hint was replayed to a replica after all 
replicas had purged their tombstones. That seems like a long shot, it would 
need one node to be down for the write and all up for the delete and for all of 
them to have purged the tombstone. But maybe we should have a max age on hints 
so it cannot happen. 

Created https://issues.apache.org/jira/browse/CASSANDRA-5379

Ensuring no hints are in place during an upgrade would work around. I tend to 
make sure hints and commit log are clear during an upgrade. 

Cheers

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 22/03/2013, at 7:54 AM, Arya Goudarzi  wrote:

> Beside the joke, would hinted handoff really have any role in this issue? I 
> have been struggling to reproduce this issue using the snapshot data taken 
> from our cluster and following the same upgrade process from 1.1.6 to 1.1.10. 
> I know snapshots only link to active SSTables. What if these returned rows 
> belong to some inactive SSTables and some bug exposed itself and marked them 
> as active? What are the possibilities that could lead to this? I am eager to 
> find our as this is blocking our upgrade.
> 
> On Tue, Mar 19, 2013 at 2:11 AM,  wrote:
> This obscure feature of Cassandra is called “haunted handoff”.
> 
>  
> 
> Happy (early) April Fools J
> 
>  
> 
> From: aaron morton [mailto:aa...@thelastpickle.com] 
> Sent: Monday, March 18, 2013 7:45 PM
> To: user@cassandra.apache.org
> Subject: Re: Lots of Deleted Rows Came back after upgrade 1.1.6 to 1.1.10
> 
>  
> 
> As you see, this node thinks lots of ranges are out of sync which shouldn't 
> be the case as successful repairs where done every night prior to the 
> upgrade. 
> 
> Could this be explained by writes occurring during the upgrade process ? 
> 
>  
> 
> I found this bug which touches timestamp and tomstones which was fixed in 
> 1.1.10 but am not 100% sure if it could be related to this issue: 
> https://issues.apache.org/jira/browse/CASSANDRA-5153
> 
> Me neither, but the issue was fixed in 1.1.0
> 
>  
> 
>  It appears that the repair task that I executed after upgrade, brought back 
> lots of deleted rows into life.
> 
> Was it entire rows or columns in a row?
> 
> Do you know if row level or column level deletes were used ? 
> 
>  
> 
> Can you look at the data in cassanca-cli and confirm the timestamps on the 
> columns make sense ?  
> 
>  
> 
> Cheers
> 
>  
> 
> -
> 
> Aaron Morton
> 
> Freelance Cassandra Consultant
> 
> New Zealand
> 
>  
> 
> @aaronmorton
> 
> http://www.thelastpickle.com
> 
>  
> 
> On 16/03/2013, at 2:31 PM, Arya Goudarzi  wrote:
> 
> 
> 
> 
> Hi,
> 
>  
> 
> I have upgraded our test cluster from 1.1.6 to 1.1.10. Followed by running 
> repairs. It appears that the repair task that I executed after upgrade, 
> brought back lots of deleted rows into life. Here are some logistics:
> 
>  
> 
> - The upgraded cluster started from 1.1.1 -> 1.1.2 -> 1.1.5 -> 1.1.6 
> 
> - Old cluster: 4 node, C* 1.1.6 with RF3 using NetworkTopology;
> 
> - Upgrade to : 1.1.10 with all other settings the same;
> 
> - Successful repairs were being done on this cluster every night;
> 
> - Our clients use nanosecond precision timestamp for cassandra calls;
> 
> - After upgrade, while running repair I say some log messages like this in 
> one node:
> 
>  
> 
> system.log.5: INFO [AntiEntropyStage:1] 2013-03-15 19:55:54,847 
> AntiEntropyService.java (line 1022) [repair 
> #0990f320-8da9-11e2--e9b2bd8ea1bd] Endpoints /XX.194.60 and /23.20.207.56 
> have 2223 range(s) out of sync for App
> 
> system.log.5: INFO [AntiEntropyStage:1] 2013-03-15 19:55:54,877 
> AntiEntropyService.java (line 1022) [repair 
> #0990f320-8da9-11e2--e9b2bd8ea1bd] Endpoints /XX.250.43 and /23.20.207.56 
> have 161 range(s) out of sync for App
> 
> system.log.5: INFO [AntiEntropyStage:1] 2013-03-15 19:55:55,097 
> AntiEntropyService.java (line 1022) [repair 
> #0990f320-8da9-11e2--e9b2bd8ea1bd] Endpoints /XX.194.60 and /23.20.250.43 
> have 2294 range(s) out of sync for App
> 
> system.log.5: INFO [AntiEntropyStage:1] 2013-03-15 19:55:59,190 
> AntiEntropyService.java (line 789) [repair 
> #0990f320-8da9-11e2--e9b2bd8ea1bd] App is fully synced (13 remaining 
> column family to sync for this session)
> 
>  
> 
> As you see, this node thinks lots of ranges are out of sync which shouldn't 
> be the case as successful repairs where done every night prior to the 
> upgrade. 
> 
>  
> 
> The App CF uses SizeTiered with gc_grace of 10 days. It has caching = 'ALL', 
> and it is fairly small (11Mb on each node).
> 
>  
> 
> I found this bug which touches timestamp and tomstones which was fixed in 
> 1.1.10 but am not 100% sure if it could be related to this issue: 
> https://issues.apache.org/jira/browse/CASSANDRA-5153
> 
>  
> 
> Any advice on how to dig 

Re: Question regarding multi datacenter and LOCAL_QUORUM

2013-03-23 Thread aaron morton
>  and read_repair_chance = 0.1
>  and dclocal_read_repair_chance = 0.0
So now I am a little confused. 

With LOCAL_QUOURM I expect the block for to be 2. And with global read repair 
active (read_reapir_chance) I expect all up nodes in all DC's to be active. 
With DC_LOCAL read repair (dclocal_read_repair_chance) I expect 3 nodes to be 
in the read. 

 So from this info

> # Cassandra Node IP=Data Center:Rack
> xx.yy.zz.143=AZ1:RAC1
> xx.yy.zz.145=AZ1:RAC1
> xx.yy.zz.146=AZ1:RAC1
> xx.yy.zz.147=AZ2:RAC2
> xx.yy.zz.148=AZ2:RAC2
> xx.yy.zz.149=AZ2:RAC2

> DEBUG [Thrift:1] 2013-03-19 00:00:53,313 ReadCallback.java (line 79) Blockfor 
> is 2; setting up requests to /xx.yy.zz.146,/xx.yy.zz.143,/xx.yy.zz.145
> 

It looks like DC_LOCAL read repair is active as all nodes from are AZ 1. 

But it should have zero possibility of running. 

Can you see any reads that involve nodes from both AZ's ? 
If you can reproduce this can you raise a bug with the logging information and 
the schema ?

Thanks


-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 22/03/2013, at 9:30 AM, Dwight Smith  wrote:

> Aaron 
> 
> Here you go:
> 
> create keyspace sipfs
>  with placement_strategy = 'NetworkTopologyStrategy'
>  and strategy_options = {AZ1 : 3, AZ2 : 3}
>  and durable_writes = true;
> 
> The CFs all have the following:
> 
> create column family xxx
>  with column_type = 'Standard'
>  and comparator = 'UTF8Type'
>  and default_validation_class = 'BytesType'
>  and key_validation_class = 'UTF8Type'
>  and read_repair_chance = 0.1
>  and dclocal_read_repair_chance = 0.0
>  and gc_grace = 864000
>  and min_compaction_threshold = 4
>  and max_compaction_threshold = 32
>  and replicate_on_write = true
>  and compaction_strategy = 
> 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy'
>  and caching = 'KEYS_ONLY'
>  and compression_options = {'sstable_compression' : 
> 'org.apache.cassandra.io.compress.SnappyCompressor'};
> 
> Regards
> 
> 
> 
> -Original Message-
> From: aaron morton [mailto:aa...@thelastpickle.com] 
> Sent: Thursday, March 21, 2013 12:47 PM
> To: user@cassandra.apache.org
> Subject: Re: Question regarding multi datacenter and LOCAL_QUORUM
> 
> Can you provide the full create keyspace statement ? 
> 
>> Yes - using NetworkTopologyStrategy
> mmm, maybe it thinks the other nodes are down. 
> 
> Cheers
> 
> -
> Aaron Morton
> Freelance Cassandra Consultant
> New Zealand
> 
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 22/03/2013, at 6:42 AM, Dwight Smith  wrote:
> 
>> Yes - using NetworkTopologyStrategy
>> 
>> From: aaron morton [mailto:aa...@thelastpickle.com]
>> Sent: Thursday, March 21, 2013 10:22 AM
>> To: user@cassandra.apache.org
>> Subject: Re: Question regarding multi datacenter and LOCAL_QUORUM
>> 
>> DEBUG [Thrift:1] 2013-03-19 00:00:53,313 ReadCallback.java (line 79) 
>> Blockfor is 2; setting up requests to 
>> /xx.yy.zz.146,/xx.yy.zz.143,/xx.yy.zz.145
>> DEBUG [Thrift:1] 2013-03-19 00:00:53,334 CassandraServer.java (line 
>> 306) get_slice DEBUG [Thrift:1] 2013-03-19 00:00:53,334 
>> ReadCallback.java (line 79) Blockfor is 2; setting up requests to 
>> /xx.yy.zz.146,/xx.yy.zz.143 DEBUG [Thrift:1] 2013-03-19 00:00:53,366 
>> CassandraServer.java (line 306) get_slice DEBUG [Thrift:1] 2013-03-19 
>> 00:00:53,367 ReadCallback.java (line 79) Blockfor is 2; setting up 
>> requests to /xx.yy.zz.146,/xx.yy.zz.143,/xx.yy.zz.145
>> This is Read Repair, as controlled by the read_repaur_chance and 
>> dclocal_read_repair_chance CF settings, in action. 
>> 
>> "Blockfor" is how many nodes the read operation is going to wait for. 
>> When the number of nodes in the request is more than blockfor it means 
>> Read Repair is active, we are reading from all UP nodes and will 
>> repair any detected differences in the background. Your read is 
>> waiting for 2 nodes to respond only (including the one we ask for the 
>> data.)
>> 
>> The odd thing here is that there are only 3 replicas nodes. Are you using 
>> the Network Topology Strategy ? If so I would expect there to be 6 nodes in 
>> the the request with RR, 3 in each DC. 
>> 
>> Cheers
>> 
>> 
>> -
>> Aaron Morton
>> Freelance Cassandra Consultant
>> New Zealand
>> 
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> -
>> Aaron Morton
>> Freelance Cassandra Consultant
>> New Zealand
>> 
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 21/03/2013, at 12:38 PM, Tyler Hobbs  wrote:
>> 
>> 
>> 
>> On Wed, Mar 20, 2013 at 3:18 PM, Tycen Stafford  wrote:
>> I don't think that's correct for a mult-dc ring, but you'll want to hear a 
>> final answer from someone more authoritative.  I could easily be wrong.  Try 
>> using the built in token generating tool (token-generator) - I don't see

Re: Incompatible Gossip 1.1.6 to 1.2.1 Upgrade?

2013-03-23 Thread aaron morton
So all nodes are 1.2 and some are still being marked as down ? 

I would try a rolling restart with -Dcassandra.load_ring_state=false added as a 
JVM _OPT in cassandra-env.sh. There is no guarantee it will fix it, but it's a 
simple thing to try. 

Cheers

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 22/03/2013, at 10:30 AM, Arya Goudarzi  wrote:

> I took Brandon's suggestion in CASSANDRA-5332 and upgraded to 1.1.10 before 
> upgrading to 1.2.2 but the issue with nodetool ring reporting machines as 
> down did not resolve. 
> 
> On Fri, Mar 15, 2013 at 6:35 PM, Arya Goudarzi  wrote:
> Thank you very much Aaron. I recall from the logs of this upgraded node to 
> 1.2.2 reported seeing others as dead. Brandon suggested in 
> https://issues.apache.org/jira/browse/CASSANDRA-5332 that I should at least 
> upgrade from 1.1.7. So, I decided to try upgrading to 1.1.10 first before 
> upgrading to 1.2.2. I am in the middle of troubleshooting some other issues I 
> had with that upgrade (posted separately), once I am done, I will give your 
> suggestion a try.
> 
> 
> On Mon, Mar 11, 2013 at 10:34 PM, aaron morton  
> wrote:
> > Is this just a display bug in nodetool or this upgraded node really sees 
> > the other ones as dead?
> Is the 1.2.2 node which is see all the others as down processing requests ?
> Is it showing the others as down in the log ?
> 
> I'm not really sure what's happening. But you can try starting the 1.2.2 node 
> with the
> 
> -Dcassandra.load_ring_state=false
> 
> parameter, append it at the bottom of the cassandra-env.sh file. It will 
> force the node to get the ring state from the others.
> 
> Cheers
> 
> -
> Aaron Morton
> Freelance Cassandra Consultant
> New Zealand
> 
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 8/03/2013, at 10:24 PM, Arya Goudarzi  wrote:
> 
> > OK. I upgraded one node from 1.1.6 to 1.2.2 today. Despite some new 
> > problems that I had and I posted them in a separate email, this issue still 
> > exists but now it is only on 1.2.2 node. This means that the nodes running 
> > 1.1.6 see all other nodes including 1.2.2 as Up. Here is the ring and 
> > gossip from nodes with 1.1.6 for example. Bold denotes upgraded node:
> >
> > Address DC  RackStatus State   Load
> > Effective-Ownership Token
> > 
> >141784319550391026443072753098378663700
> > XX.180.36us-east 1b  Up Normal  49.47 GB25.00%  
> > 1808575600
> > XX.231.121  us-east 1c  Up Normal  47.08 GB25.00%   
> >7089215977519551322153637656637080005
> > XX.177.177  us-east 1d  Up Normal  33.64 GB25.00%   
> >14178431955039102644307275311465584410
> > XX.7.148us-east 1b  Up Normal  41.27 GB25.00%   
> >42535295865117307932921825930779602030
> > XX.20.9 us-east 1c  Up Normal  38.51 GB25.00%   
> >49624511842636859255075463585608106435
> > XX.86.255us-east 1d  Up Normal  34.78 GB25.00%  
> > 56713727820156410577229101240436610840
> > XX.63.230us-east 1b  Up Normal  38.11 GB25.00%  
> > 85070591730234615865843651859750628460
> > XX.163.36   us-east 1c  Up Normal  44.25 GB25.00%   
> >92159807707754167187997289514579132865
> > XX.31.234us-east 1d  Up Normal  44.66 GB25.00%  
> > 99249023685273718510150927169407637270
> > XX.132.169   us-east 1b  Up Normal  44.2 GB 25.00%  
> > 127605887595351923798765477788721654890
> > XX.71.63 us-east 1c  Up Normal  38.74 GB25.00%  
> > 134695103572871475120919115443550159295
> > XX.197.209  us-east 1d  Up Normal  41.5 GB 25.00%   
> >141784319550391026443072753098378663700
> >
> > /XX.71.63
> >   RACK:1c
> >   SCHEMA:99dce53b-487e-3e7b-a958-a1cc48d9f575
> >   LOAD:4.1598705272E10
> >   DC:us-east
> >   INTERNAL_IP:XX.194.92
> >   STATUS:NORMAL,134695103572871475120919115443550159295
> >   RPC_ADDRESS:XX.194.92
> >   RELEASE_VERSION:1.1.6
> > /XX.86.255
> >   RACK:1d
> >   SCHEMA:99dce53b-487e-3e7b-a958-a1cc48d9f575
> >   LOAD:3.734334162E10
> >   DC:us-east
> >   INTERNAL_IP:XX.6.195
> >   STATUS:NORMAL,56713727820156410577229101240436610840
> >   RPC_ADDRESS:XX.6.195
> >   RELEASE_VERSION:1.1.6
> > /XX.7.148
> >   RACK:1b
> >   SCHEMA:99dce53b-487e-3e7b-a958-a1cc48d9f575
> >   LOAD:4.4316975808E10
> >   DC:us-east
> >   INTERNAL_IP:XX.47.250
> >   STATUS:NORMAL,42535295865117307932921825930779602030
> >   RPC_ADDRESS:XX.47.250
> >   RELEASE_VERSION:1.1.6
> > /XX.63.230
> >   RACK:1b
> >   SCHEMA:99dce53b-487e

Re: Recovering from a faulty cassandra node

2013-03-23 Thread aaron morton
With thins like cleanup (and upgradesstables) I then to run them on every RF'th 
node. 

Cheers

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 22/03/2013, at 10:59 AM, Jabbar Azam  wrote:

> nodetool cleanup command removes keys which can be deleted from the node the  
> command is run. So I'm assuming I can run nodetool cleanup on all the old 
> nodes in parallel. Wouldn't do this on a live cluster as it's I/O intensive 
> on each node. 
> 
> 
> On 21 March 2013 17:26, Jabbar Azam  wrote:
> Can I do a multiple node nodetool cleanup on my test cluster?
> 
> On 21 Mar 2013 17:12, "Jabbar Azam"  wrote:
> 
> All cassandra-topology.properties are the same.
> 
> The node add appears to be successful. I can see it using nodetool status. 
> I'm doing a node cleanup on the old nodes and then will do a node remove, to 
> remove the old node. The actual node join took about 6 hours. The wiped 
> node(now new node) has about 324 GB of files in /var/lib/cassandra
> 
> 
> 
> 
> 
> On 21 March 2013 16:58, aaron morton  wrote:
>>  Not sure if I needed to change cassandra-topology.properties file on the 
>> existing nodes.
> If you are using the PropertyFileSnitch all nodes need to have the same 
> cassandra-topology.properties file. 
> 
> Cheers
> 
> -
> Aaron Morton
> Freelance Cassandra Consultant
> New Zealand
> 
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 21/03/2013, at 1:34 AM, Jabbar Azam  wrote:
> 
>> I've added the node with a different IP address and after disabling the 
>> firewall data is being streamed from the existing nodes to the wiped node. 
>> I'll do a cleanup, followed by remove node once it's done.
>> 
>> I've also added the new node to the existing nodes' 
>> cassandra-topology.properties file and restarted them. I also found I had 
>> iptables switched on and couldn't understand why the wiped node couldn't see 
>> the cluster. Not sure if I needed to change cassandra-topology.properties 
>> file on the existing nodes.
>> 
>> 
>> 
>> 
>> On 19 March 2013 15:49, Jabbar Azam  wrote:
>> Do I use removenode before adding the reinstalled node or after?
>> 
>> 
>> On 19 March 2013 15:45, Alain RODRIGUEZ  wrote:
>> In 1.2, you may want to use the nodetool removenode if your server i broken 
>> or unreachable, else I guess nodetool decommission remains the good way to 
>> remove a node. (http://www.datastax.com/docs/1.2/references/nodetool)
>> 
>> When this node is out, rm -rf /yourpath/cassandra/* on this serveur, change 
>> the configuration if needed (not sure about the auto_bootstrap param) and 
>> start Cassandra on that node again. It should join the ring as a new node.
>> 
>> Good luck.
>> 
>> 
>> 2013/3/19 Hiller, Dean 
>> 
>> Since you "cleared" out that node, it IS the replacement node.
>> 
>> Dean
>> 
>> From: Jabbar Azam mailto:aja...@gmail.com>>
>> Reply-To: "user@cassandra.apache.org" 
>> mailto:user@cassandra.apache.org>>
>> Date: Tuesday, March 19, 2013 9:29 AM
>> To: "user@cassandra.apache.org" 
>> mailto:user@cassandra.apache.org>>
>> Subject: Re: Recovering from a faulty cassandra node
>> 
>> Hello Dean.
>> 
>> I'm using vnodes so can't specify a token. In addition I can't follow the 
>> replace node docs because I don't have a replacement node.
>> 
>> 
>> On 19 March 2013 15:25, Hiller, Dean 
>> mailto:dean.hil...@nrel.gov>> wrote:
>> I have not done this as of yet but from all that I have read your best 
>> option is to follow the replace node documentation which I belive you need to
>> 
>> 
>>  1.  Have the token be the same BUT add 1 to it so it doesn't think it's the 
>> same computer
>>  2.  Have the bootstrap option set or something so streaming takes affect.
>> 
>> I would however test that all out in QA to make sure it works and if you 
>> have QUOROM reads/writes a good part of that test would be to take node X 
>> down after your node Y is back in the cluster to make sure reads/writes are 
>> working on the node you fixed…..you just need to make sure node X shares one 
>> of the token ranges of node Y AND your writes/reads are in that token range.
>> 
>> Dean
>> 
>> From: Jabbar Azam 
>> mailto:aja...@gmail.com>>>
>> Reply-To: 
>> "user@cassandra.apache.org>"
>>  
>> mailto:user@cassandra.apache.org>>>
>> Date: Tuesday, March 19, 2013 8:51 AM
>> To: 
>> "user@cassandra.apache.org>"
>>  
>> mailto:user@cassandra.apache.org>>>
>> Subject: Recovering from a faulty cassandra node
>> 
>> Hello,
>> 
>> I am using Cassandra 1.2.2 on a 4 node 

Re: Unable to fetch large amount of rows

2013-03-23 Thread aaron morton
> [pp] no, I didn’t look at proxyhistogram, in fact I don’t know how to run it. 
> Can you give me insights of how to run it?
It's available on nodetool but I cannot remember the version it was added. 

If it's not there the information has always been available on the 
StorageProxyMBean. 

Cheers

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 22/03/2013, at 5:15 PM, Pushkar Prasad  
wrote:

> Answers prefixed with [PP]
> From: aaron morton [mailto:aa...@thelastpickle.com] 
> Sent: 21 March 2013 23:11
> To: user@cassandra.apache.org
> Subject: Re: Unable to fetch large amount of rows
>  
> + Did run cfhistograms, the results are interesting (Note: row cache is
> disabled):
> SSTables in cfhistograms is a friend here. It tells you how many sstables 
> were read from per read, if it's above 3 I then take a look at the data 
> model. If you case I would be wondering how long that row with the time stamp 
> is written to. Is it spread over many sstables ? 
>  
> [PP] Just one SSTable
>>+ 75% time is spent on disk latency
> Do you  mean 75% of the latency reported by proxyhistorgrams is also reported 
> by cfhistograms
>  
> [pp] no, I didn’t look at proxyhistogram, in fact I don’t know how to run it. 
> Can you give me insights of how to run it?
>> +++ When query made on node on which all the records are not present
> Do you mean the co-ordinator for the request was not a replica for the row?
>  
> [PP] Correct
>>+ If my query is 
>> 
>>-   select * from schema where timestamp = '..' ORDER BY MacAddress,
>> would that be faster than, say
>> 
>>-   select * from schema where timestamp = '..' 
> As usual in a DB, it's faster to not re-order things. I'd have to check if 
> the order by will no-op if it's the same as the clustering columns, for now 
> lets just keep it out. 
>  
>>  
>> 2) Why does response time suffer when query is made on a node on which
>> records to be returned are not present? In order to be able to get better
>> response when queried from a different node, can something be done?
> During a read one node is asked to return the data, and the others to return 
> a digest of their data. When the read runs on a node that is a replica the 
> data read is done locally and the others are asked for a digest, this can 
> lead to better performance. If you are asking for a large row this will have 
> a larger impact. 
>  
> Astyanax can direct reads to nodes which are replicas. 
>  
> Cheers
>  
>  
> -
> Aaron Morton
> Freelance Cassandra Consultant
> New Zealand
>  
> @aaronmorton
> http://www.thelastpickle.com
>  
> On 21/03/2013, at 4:48 PM, Pushkar Prasad 
>  wrote:
> 
> 
> Yes, I'm reading from a single partition.
> 
> -Original Message-
> From: Hiller, Dean [mailto:dean.hil...@nrel.gov] 
> Sent: 21 March 2013 01:38
> To: user@cassandra.apache.org
> Subject: Re: Unable to fetch large amount of rows
> 
> Is your use case reading from a single partition?  If so, you may want to
> switch to something like playorm which does virtual partitions so you still
> get the performance of multiple disks when reading from a single partition.
> My understanding is a single cassandra partition exists on a single node.
> Anyways, just an option if that is your use-case.
> 
> Later,
> Dean
> 
> From: Pushkar Prasad
> mailto:pushkar.prasad@airtightnetworks.
> net>>
> Reply-To: "user@cassandra.apache.org"
> mailto:user@cassandra.apache.org>>
> Date: Wednesday, March 20, 2013 11:41 AM
> To: "user@cassandra.apache.org"
> mailto:user@cassandra.apache.org>>
> Subject: RE: Unable to fetch large amount of rows
> 
> Hi aaron.
> 
> I added pagination, and things seem to have started performing much better.
> With 1000 page size, now able to fetch 500K records in 25-30 seconds.
> However, I'd like to point you to some interesting observations:
> 
> + Did run cfhistograms, the results are interesting (Note: row cache is
> disabled):
> +++ When query made on node on which all the records are present
>+ 75% time is spent on disk latency
>+ Example: When 50 K entries were fetched, it took 2.65 seconds, out
> of which 1.92 seconds were spent in disk latency
> +++ When query made on node on which all the records are not present
>+ Considerable amount of time is spent on things other than disk
> latency (probably deserialization/serialization, network, etc.)
>+ Example: When 50 K entries were fetched, it took 5.74 seconds, out
> of which 2.21 seconds were spent in disk latency.
> 
> I've used Astyanax to run the above queries. The results were same when run
> with different data points. Compaction has not been done after data
> population yet.
> 
> I've a few questions:
> 1) Is it necessary to fetch the records in natural order of comparator
> column in order to get a high throughput? I'm trying to fetch all

Re: create secondary index on column family

2013-03-23 Thread aaron morton
> But a error is thrown saying "can not parse name as hex bytes".
If the comparator is Bytes then the column names need to be a hex string. 

The easiest thing to do is create a CF where the comparator is UTF8Type so you 
can use string column names. 

> just that the UTF8Type needs to be validated before storing the data into 
> database and BytesType need not to?

It takes *very* little additional effort. 

Cheers

 
-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 23/03/2013, at 12:10 AM, Xu Renjie  wrote:

> Sorry, continued:
>I have created a column family User with no parameters specified, just
>  create column family User.
>  Then I checked that the default comparator is BytesType. 
>  
>   Then I want to create secondary index on one column like below:
>   update column family User with column_metadata=[{column_name:name, 
> validation_class:BytesType, index_type:0}];
> But a error is thrown saying "can not parse name as hex bytes".
> 
> So I wonder under this situation, is it possible to create index using 
> cassandra-cli, if possible, how?
> 
> Furthermore, I wonder what's the difference of type BytesType and UTF8Type 
> and other types underlying.
> If I store string 'name' into database, do they have the same internal bytes 
> stored in Cassandra,
> just that the UTF8Type needs to be validated before storing the data into 
> database and BytesType need not to?
> 
> 
> On Fri, Mar 22, 2013 at 7:00 PM, Xu Renjie  wrote:
> Hello, guys:
>I am new to Cassandra. I am currently using cassandra-cli(version 1.1.6). 
> 



Re: Thrift key

2013-03-23 Thread aaron morton
> When I used Thrift, the key is changed by Cassandra. Cassandra Thrift add a 
> prefix control bytes to Partitioner key and send the object always to the 
> same machine because the key prefix is the same.
Where do you think this is happening in the code?

Remember the key received by thrift is transformed into a token using the 
partitioner and the token is used to place the row, not the key. 

Cheers

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 23/03/2013, at 2:31 AM, Sávio Teles  wrote:

> I'm running a Cassandra 1.1.10 cluster with a ByteOrderedPartitioner. I'm 
> generating a key to force an object to be stored in a specific machine. When 
> I used org.apache.cassandra.thrift.CassandraServer to store the object, this 
> object was stored on correct machine. When I used Thrift, the key is changed 
> by Cassandra. Cassandra Thrift add a prefix control bytes to Partitioner key 
> and send the object always to the same machine because the key prefix is the 
> same.
> 
> Is it any error? 
> 
> -- 
> Atenciosamente,
> Sávio S. Teles de Oliveira
> voice: +55 62 9136 6996
> http://br.linkedin.com/in/savioteles
> Mestrando em Ciências da Computação - UFG 
> Arquiteto de Software
> Laboratory for Ubiquitous and Pervasive Applications (LUPA) - UFG



Re: Cassandra - conflict resolution for column updates with identical timestamp

2013-03-23 Thread Edward Capriolo
The value that sorts higher, this way it is deterministic.

On Sat, Mar 23, 2013 at 12:12 PM, dong.yajun  wrote:

> Hello,
>
> I would like to know which write wins in case of two updates with the
> same client timestamp in Cassandra.
>
> Initial data: KeyA: { col1:"val AA", col2:"val BB", col3:"val CC"}
>
> Client 1 sends update: KeyA: { col1:"val C1", col2:"val B1"} on Sx
>
> Client 2 sends update: KeyA: { col1:"val C2", col2:"val B2"} on Sy
>
> Both updates have the same timestamp.
>
> --
> *Ric Dong*
>
>


Re: Stream fails during repair, two nodes out-of-memory

2013-03-23 Thread Dane Miller
On Fri, Mar 22, 2013 at 5:58 PM, Wei Zhu  wrote:
> compaction needs some disk I/O. Slowing down our compaction will improve 
> overall
> system performance. Of course, you don't want to go too slow and fall behind 
> too much.

Hmm.  Even after making the suggested configuration changes, repair
still fails with OOM (but only one node died this time, which is an
improvement).  It looks like we hit OOM when repair starts streaming
multiple cfs simultaneously.  Just prior to OOM, the node loses
contact with another node in the cluster and starts storing hints.

I'm wondering if I should throttle streaming, and/or repair only one
CF at a time.

> From: "Dane Miller"
> Subject: Re: Stream fails during repair, two nodes out-of-memory
>
> On Thu, Mar 21, 2013 at 10:28 AM, aaron morton  
> wrote:
>> heap of 1867M is kind of small. According to the discussion on this list,
>> it's advisable to have m1.xlarge.
>>
>> +1
>>
>> In cassadrea-env.sh set the MAX_HEAP_SIZE to 4GB, and the NEW_HEAP_SIZE to
>> 400M
>>
>> In the yaml file set
>>
>> in_memory_compaction_limit_in_mb to 32
>> compaction_throughput_mb_per_sec to 8
>> concurrent_compactors to 2
>>
>> This will slow down compaction a lot. You may want to restore some of these
>> settings once you have things stable.
>>
>> You have an under powered box for what you are trying to do.
>
> Thanks very much for the info.  Have made the changes and am retrying.
>  I'd like to understand, why does it help to slow compaction?
>
> It does seem like the cluster is under powered to handle our
> application's full write load plus repairs, but it operates fine
> otherwise.
>
> On Wed, Mar 20, 2013 at 8:47 PM, Wei Zhu  wrote:
>> It's clear you are out of memory. How big is your data size?
>
> 120 GB per node, of which 50% is actively written/updated, and 50% is
> read-mostly.
>
> Dane
>


Observation on shuffling vs adding/removing nodes

2013-03-23 Thread Andrew Bialecki
Just curious if anyone has any thoughts on something we've observed in a
small test cluster.

We had around 100 GB of data on a 3 node cluster (RF=2) and wanted to start
using vnodes. We upgraded the cluster to 1.2.2 and then followed the
instructions for using vnodes. We initially tried to run a shuffle, however
it seemed to be going really slowly (very little progress by watching
"cassandra-shuffle ls | wc -l" after 5-6 hours and no errors in logs), so
we cancelled it and instead added 3 nodes to the cluster, waited for them
to bootstrap, and then decommissioned the first 3 nodes. Total process took
about 3 hours. My assumption is that the final result is the same in terms
of data distributed somewhat randomly across nodes now (assuming no bias in
the token ranges selected when bootstrapping a node).

If that assumption is correct, the observation would be, if possible,
adding nodes and then removing nodes appears to be a faster way to shuffle
data for small clusters. Obviously not always possible, but I thought I'd
just throw this out there in case anyone runs into a similar situation.
This cluster is unsurprisingly on EC2 instances, which made provisioning
and shutting down nodes extremely easy.

Cheers,
Andrew


Re: High disk I/O during reads

2013-03-23 Thread Matt Kap
Having battled similar issues with read latency recently, here's some
general things to look out for.

- At 118ms, something is definitely broken. You should be looking at
under 10ms or lower, depending on hardware.
- Do "nodetool info" on all 5 nodes. Is the load distributed evenly?
Is it reasonable (under 500GB)?
- Make sure you aren't running low on heap space. You could see that
from "nodetool info" also. If you are running low, very bad things
begin to happen (lots of GC, constant flushing of Memtables, reduction
of Key Cache, etc). Generally, once there, the node doesn't recover,
and read latency goes to sh*t.
- Which compaction strategy are you using? Leveled compactions or
size-tiered? There's different issues with both.
- Is your Key Cache turned on? What's the Key Cache hit rate?
- Is the Read Latency the same on all nodes? Or just one in particular?
- Are pending compactions building up?
- What's %util on disk? Same on all nodes?

I would go through "nodetool cfstats, info, compactionstats, tpstats",
and see if things are roughly the same across all the nodes. You could
also just be under capacity, but more likely, there's an actual
problem looming somewhere.

Cheers!
-Matt

On Sat, Mar 23, 2013 at 3:18 AM,   wrote:
> You can try to disable readahead on cassandra data disk.
>
> Jon Scarborough  написал(а):
>>
>> Checked tpstats, there are very few dropped messages.
>>
>> Checked histograms. Mostly nothing surprising. The vast majority of rows
>> are small, and most reads only access one or two SSTables.
>>
>> What I did discover is that of our 5 nodes, one is performing well, with
>> disk I/O in the ballprk that seems reasonable. The other 4 nodes are doing
>> roughly 4x the disk i/O per second.  Interestingly, the node that is
>> performing well also seems to be servicing about twice the number of reads
>> that the other nodes are.
>>
>> I compared configuration between the node performing well to those that
>> aren't, and so far haven't found any discrepancies.
>>
>> On Fri, Mar 22, 2013 at 10:43 AM, Wei Zhu  wrote:
>>>
>>> According to your cfstats, read latency is over 100 ms which is really
>>> really slow. I am seeing less than 3ms reads for my cluster which is on SSD.
>>> Can you also check the nodetool cfhistorgram, it tells you more about the
>>> number of SSTable involved and read/write latency. Somtimes average doesn't
>>> tell you the whole storey.
>>> Also check your nodetool tpstats, are there a lot dropped reads?
>>>
>>> -Wei
>>> - Original Message -
>>> From: "Jon Scarborough" 
>>> To: user@cassandra.apache.org
>>> Sent: Friday, March 22, 2013 9:42:34 AM
>>> Subject: Re: High disk I/O during reads
>>>
>>> Key distribution across probably varies a lot from row to row in our
>>> case. Most reads would probably only need to look at a few SSTables, a few
>>> might need to look at more.
>>>
>>> I don't yet have a deep understanding of C* internals, but I would
>>> imagine even the more expensive use cases would involve something like this:
>>>
>>> 1) Check the index for each SSTable to determine if part of the row is
>>> there.
>>> 2) Look at the endpoints of the slice to determine if the data in a
>>> particular SSTable is relevant to the query.
>>> 3) Read the chunks of those SSTables, working backwards from the end of
>>> the slice until enough columns have been read to satisfy the limit clause in
>>> the query.
>>>
>>> So I would have guessed that even the more expensive queries on wide rows
>>> typically wouldn't need to read more than a few hundred KB from disk to do
>>> all that. Seems like I'm missing something major.
>>>
>>> Here's the complete CF definition, including compression settings:
>>>
>>> CREATE COLUMNFAMILY conversation_text_message (
>>> conversation_key bigint PRIMARY KEY
>>> ) WITH
>>> comment='' AND
>>>
>>> comparator='CompositeType(org.apache.cassandra.db.marshal.DateType,org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.AsciiType,org.apache.cassandra.db.marshal.AsciiType)'
>>> AND
>>> read_repair_chance=0.10 AND
>>> gc_grace_seconds=864000 AND
>>> default_validation=text AND
>>> min_compaction_threshold=4 AND
>>> max_compaction_threshold=32 AND
>>> replicate_on_write=True AND
>>> compaction_strategy_class='SizeTieredCompactionStrategy' AND
>>>
>>> compression_parameters:sstable_compression='org.apache.cassandra.io.compress.SnappyCompressor';
>>>
>>> Much thanks for any additional ideas.
>>>
>>> -Jon
>>>
>>>
>>>
>>> On Fri, Mar 22, 2013 at 8:15 AM, Hiller, Dean < dean.hil...@nrel.gov >
>>> wrote:
>>>
>>>
>>> Did you mean to ask "are 'all' your keys spread across all SSTables"? I
>>> am guessing at your intention.
>>>
>>> I mean I would very well hope my keys are spread across all sstables or
>>> otherwise that sstable should not be there as he has no keys in it ;).
>>>
>>> And I know we had HUGE disk size from the duplication in our sstables on
>>> size-tiered compactionwe never ran a major compaction but after we
>>> swi

Re: Cassandra - conflict resolution for column updates with identical timestamp

2013-03-23 Thread dong.yajun
Thanks Capriolo,

Umm.. so is there any background or history abort this issue?

On Sun, Mar 24, 2013 at 2:32 AM, Edward Capriolo wrote:

> The value that sorts higher, this way it is deterministic.
>
>
> On Sat, Mar 23, 2013 at 12:12 PM, dong.yajun  wrote:
>
>> Hello,
>>
>> I would like to know which write wins in case of two updates with the
>> same client timestamp in Cassandra.
>>
>> Initial data: KeyA: { col1:"val AA", col2:"val BB", col3:"val CC"}
>>
>> Client 1 sends update: KeyA: { col1:"val C1", col2:"val B1"} on Sx
>>
>> Client 2 sends update: KeyA: { col1:"val C2", col2:"val B2"} on Sy
>>
>> Both updates have the same timestamp.
>>
>> --
>> *Ric Dong*
>>
>>
>


-- 
*Ric Dong *
Newegg Ecommerce, MIS department