Re: How safe is nodetool move in 1.2 ?

2014-04-16 Thread Richard Low
On 16 April 2014 05:08, Jonathan Lacefield jlacefi...@datastax.com wrote:

 Assuming you have enough nodes not undergoing move to meet your CL
 requirements, then yes, your cluster will still accept reads and writes.
 However, it's always good to test this before doing it in production to
 ensure your cluster and app will function as designed.


This is not a correctness requirement: writes go to the move source and
destination during the move and reads come from the source. Otherwise you
could lose data during move (and certainly would lose data if replication
factor was one). However, nodes that are involved in the move will be
slower so it will be better for performance to not move nodes that share
replicas simultaneously.

Richard.


Re: Vnodes and replication

2014-04-08 Thread Richard Low
On 8 April 2014 09:29, vck veesee...@gmail.com wrote:

  After reading through the vnodes and partitioning described in the
 datastax documentation, I am still confused about how rows are
 partitioned/replicated.

 With vnodes, I know that each Node on the ring now supports many token
 ranges per Node. However I am still not very clear on how the replication
 is carried out.

 If I have 4 Nodes A, B, C, D and their ranges are below.

 A - [1,2,3,4]
 B - [5,6,7,8]
 C - [9,10,11,12]
 D - [13,14,15,16]

 After a shuffle, lets assume the Nodes are shuffled like this.

 A - [1,5,13,9]
 B - [2,6,14,10]
 C - [3,7,15,11]
 D - [4,8,16,12]

 now if I use a simple replication strategy and RF=3, the first replica
 rowkey gets placed on node determined by the partitioner.

 Prior to vnode, if my rowkey hash is 2, it gets placed on Nodes A, B, C.
 With the above vnode setup, if my rowkey hash is 2, does it get placed in
 B, C, D ?

Yes. The only difference vnodes makes to the calculation of where replicas
live is that replicas are not placed on the same node.

Richard.


Re: Denial of Service Issue

2013-10-11 Thread Richard Low
On 11 October 2013 14:03, thorsten.s...@t-systems.com wrote:

  I found the issue below concerning inactive client connections (see 
 *Cassandra
 Security*http://jkb.netii.net/index.php/pub/sinosqldb/cassandra-security).
 We are using Cassandra 1.2.4 and the Cassandra JDBC driver as client. Is
 this still an existing issue?
 Quoted from site above:
 Denial of Service problem:
 Cassandra uses a Thread- Per-Client model in its network code. Since
 setting up a connection requires the Cassandra server to start a new thread
 on each connection (in addition to the TCP overhead incurred by the
 network), the Cassandra project recommends utilizing some sort of
 connection pooling. An attacker can prevent the Cassandra server from
 accepting new client connections by causing the Cassandra server to
 allocate all its resources to fake connection attempts. The only pieces of
 information required by an attacker are the IP addresses of the cluster
 members, and this information can be obtained by passively sniffing the
 network. The current implementation doesn’t timeout inactive connections,
 so any connection that is opened without actually passing data consumes a
 thread and a file-descriptor that are never released.


 This is still an issue, but you must not expose Cassandra to untrusted
users.  Just like you wouldn't let untrusted users have network access to
your Oracle, MySQL, etc. servers.

Richard.


Re: nodetool cfhistograms refresh

2013-10-01 Thread Richard Low
On 1 October 2013 16:21, Rene Kochen rene.koc...@schange.com wrote:

 Quick question.

 I am using Cassandra 1.0.11

 When is nodetool cfhistograms output reset? I know that data is collected
 during read requests. But I am wondering if it is data since the beginning
 (start of Cassandra) or if it is reset periodically?


It is reset on node restart and on each call to nodetool cfhistograms.

Richard.


Re: Cassandra 1.2.9 cluster with vnodes is heavily unbalanced.

2013-09-19 Thread Richard Low
On 19 September 2013 02:06, Jayadev Jayaraman jdisal...@gmail.com wrote:

We use vnodes with num_tokens = 256 ( 256 tokens per node ) . After loading
 some data with sstableloader , we find that the cluster is heavily
 imbalanced :


How did you select the tokens?  Is this a brand new cluster which started
on first boot with num_tokens = 256 and chose random tokens?  Or did you
start with num_tokens = 1 and then increase it?

Richard.


Re: Row size in cfstats vs cfhistograms

2013-09-19 Thread Richard Low
On 19 September 2013 10:31, Rene Kochen rene.koc...@schange.com wrote:

I use Cassandra 1.0.11

 If I do cfstats for a particular column family, I see a Compacted row
 maximum size of 43388628

 However, when I do a cfhistograms I do not see such a big row in the Row
 Size column. The biggest row there is 126934.

 Can someone explain this?


The 'Row Size' column is showing the number of rows that have a size
indicated by the value in the 'Offset' column.  So if your output is like

Offset  Row Size
1131752  10
1358102  100

It means you have 100 rows with size between 1131752 and 1358102 bytes.  It
doesn't mean there are rows of size 100.

Richard.


Re: Cassandra 1.2.9 cluster with vnodes is heavily unbalanced.

2013-09-19 Thread Richard Low
I think what has happened is that Cassandra was started with num_tokens =
1, then shutdown and num_tokens set to 256.  When this happens, the first
time Cassandra chooses a single random token.  Then when restarted it
splits the token into 256 adjacent ranges.

You can see something like this has happened because the tokens for each
node are sequential.

The way to fix it is to, assuming you don't want the data, shutdown your
cluster, wipe the whole data and commitlog directories, then start
Cassandra again.

Richard.


On 19 September 2013 13:16, Suruchi Deodhar 
suruchi.deod...@generalsentiment.com wrote:

 Hi Richard,
 This is a brand new cluster which started with num_tokens =256 on first
 boot and chose random tokens. The attached ring status is after data is
 loaded into the cluster for the first time using sdtableloader and remains
 that way even after Cassandra is restarted.

 Thanks,
 Suruchi

 On Sep 19, 2013, at 3:46, Richard Low rich...@wentnet.com wrote:

 On 19 September 2013 02:06, Jayadev Jayaraman jdisal...@gmail.com wrote:

  We use vnodes with num_tokens = 256 ( 256 tokens per node ) . After
 loading some data with sstableloader , we find that the cluster is heavily
 imbalanced :


 How did you select the tokens?  Is this a brand new cluster which started
 on first boot with num_tokens = 256 and chose random tokens?  Or did you
 start with num_tokens = 1 and then increase it?

 Richard.




Re: Cassandra 1.2.9 cluster with vnodes is heavily unbalanced.

2013-09-19 Thread Richard Low
The only thing you need to guarantee is that Cassandra doesn't start with
num_tokens=1 (the default in 1.2.x) or, if it does, that you wipe all the
data before starting it with higher num_tokens.


On 19 September 2013 19:07, Robert Coli rc...@eventbrite.com wrote:

 On Thu, Sep 19, 2013 at 10:59 AM, Suruchi Deodhar 
 suruchi.deod...@generalsentiment.com wrote:

 Do you suggest I should try with some other installation mechanism? Are
 there any known problems with the tar installation of cassandra 1.2.9 that
 I should be aware of?


 I was asking in the context of this JIRA :

 https://issues.apache.org/jira/browse/CASSANDRA-2356

 Which does not seem to apply in your case!

 =Rob



Re: Cassandra 1.2.9 cluster with vnodes is heavily unbalanced.

2013-09-19 Thread Richard Low
On 19 September 2013 20:36, Suruchi Deodhar 
suruchi.deod...@generalsentiment.com wrote:

 Thanks for your replies. I wiped out my data from the cluster and also
 cleared the commitlog before restarting it with num_tokens=256. I then
 uploaded data using sstableloader.

 However, I am still not able to see a uniform distribution of data across
 nodes of the clusters.

 The output of the bin/nodetool -h localhost status commands looks like
 follows. Some nodes have data as low as 1.12MB while some have as high as
 912.57 MB.


Now the 'Owns (effective)' column is showing the tokens are roughly
balanced.  So now the problem is the data isn't uniform - either you have
some rows much larger than others or some nodes are missing data that could
be replicated by running repair.

Richard.


Re: w00tw00t.at.ISC.SANS.DFind not found

2013-09-08 Thread Richard Low
On 8 September 2013 02:55, Tim Dunphy bluethu...@gmail.com wrote:

 Hey all,

  I'm seeing this exception in my cassandra logs:

 Exception during http request
 mx4j.tools.adaptor.http.HttpException: file
 mx4j/tools/adaptor/http/xsl/w00tw00t.at.ISC.SANS.DFind:) not found
 at
 mx4j.tools.adaptor.http.XSLTProcessor.notFoundElement(XSLTProcessor.java:314)
 at
 mx4j.tools.adaptor.http.HttpAdaptor.findUnknownElement(HttpAdaptor.java:800)
 at
 mx4j.tools.adaptor.http.HttpAdaptor$HttpClient.run(HttpAdaptor.java:976)

 Do I need to be concerned about the security of this server? How can I
 correct/eliminate this error message? I've just upgraded to Cassandra 2.0
 ,and this is the first time I've seen this error.


There is a web vulnerability scanner that does GET
/w00tw00t.at.ISC.SANS.DFind:) on anything it thinks is HTTP.  This
probably means your mx4j port is open to the public which is a security
issue.  This means anyone can e.g. delete all your data or stop your
Cassandra nodes.  You should make sure that all your Cassandra ports (at
least) are firewalled so only you and other nodes can connect.

Richard.


Re: successful use of shuffle?

2013-09-02 Thread Richard Low
On 30 August 2013 18:42, Jeremiah D Jordan jeremiah.jor...@gmail.comwrote:

You need to introduce the new vnode enabled nodes in a new DC.  Or you
 will have similar issues to
 https://issues.apache.org/jira/browse/CASSANDRA-5525

 Add vnode DC:

 http://www.datastax.com/documentation/cassandra/1.2/webhelp/index.html#cassandra/operations/ops_add_dc_to_cluster_t.html

 Point clients to new DC

 Remove non vnode DC:

 http://www.datastax.com/documentation/cassandra/1.2/webhelp/index.html#cassandra/operations/ops_decomission_dc_t.html


This is a good workaround if you have the hardware to temporarily have a
cluster that's double the size.  If you don't then I think shuffle is the
only option, but it is known to have issues.

Richard.


Re: How many seed nodes should I use?

2013-08-29 Thread Richard Low
On 29 August 2013 01:55, Ike Walker ike.wal...@flite.com wrote:

 What is the best practice for how many seed nodes to have in a Cassandra
 cluster? I remember reading a recommendation of 2 seeds per datacenter in
 Datastax documentation for 0.7, but I'm interested to know what other
 people are doing these days, especially in AWS.

 I'm running a cluster of 12 nodes at AWS. Each node runs Cassandra 1.2.5
 on an m1.xlarge EC2 instance, and they are spread across 3 availability
 zones within a single region.

 To keep things simple I currently have all 12 nodes listed as seeds. That
 seems like overkill to me, but I don't know the pros and cons of too many
 or too few seeds.


Seeds are used for bootstrapping a new node so it can discover the others.
 Existing nodes store a list of the other nodes it has seen so doesn't need
the seeds each time it starts up.  Seeds are treated slightly differently
in gossip though to ensure a node keeps trying to connect to seeds in case
of a partition.

The best recommendations are to use the same seed list on each node and
just a few.  More than your replication factor is almost certainly too
many, but the cost of too many is very little.

Richard.


Re: token(), limit and wide rows

2013-08-17 Thread Richard Low
You can do it by using two types of query.  One using token as you suggest,
the other by fixing the partition key and walking through the other parts
of the composite primary key.

For example, consider the table:

create table paging (a text, b text, c text primary key (a, b));

I inserted ('1', '1', 'x'), ('1', '2', 'x'), ..., ('1', '5', 'x') and then
again for a='2.  Suppose the paging size is 3, then start with

 select * from paging limit 3;

 a | b | c
---+---+---
 2 | 1 | x
 2 | 2 | x
 2 | 3 | x

Now you don't know if there are more items with a='2', so run:

 select * from paging where a = '2' and b  '3' limit 3;

 a | b | c
---+---+---
 2 | 4 | x
 2 | 5 | x

You know there aren't any more because only two results were obtained, but
you can continue with greater values of b if required.

Now move on to the next a value (in token order):

 select * from paging where token(a)  token('2') limit 3;

 a | b | c
---+---+---
 1 | 1 | x
 1 | 2 | x
 1 | 3 | x

and so on.

I don't know if there is any client library support for this, but it would
be useful.  But I think in Cassandra 2.0, CASSANDRA-4415 and CASSANDRA-4536
will solve this.

Richard.

On 16 August 2013 17:16, Jonathan Rhone jonat...@shareablee.com wrote:

 Read

 http://www.datastax.com/dev/blog/cql3-table-support-in-hadoop-pig-and-hive

 And look at


 http://fossies.org/dox/apache-cassandra-1.2.8-src/CqlPagingRecordReader_8java_source.html

 - Jon


 On Fri, Aug 16, 2013 at 12:08 PM, Keith Freeman 8fo...@gmail.com wrote:

 I've run into the same problem, surprised nobody's responded to you.  Any
 time someone asks how do I page through all the rows of a table in CQL3?,
 the standard answer is token() and limit.  But as you point out, this
 method will often miss some data from wide rows.

 Maybe a Cassandra expert will chime in if we're wrong.

 Your suggestion is possible if you know how to find the previous value of
 'name' field (and are willing to filter out repeated rows), but wouldn't
 that be difficult/impossible with some keys?  So then, is there a way to do
 paging queries that get ALL of the rows, even in wide rows?



 On 08/13/2013 02:46 PM, Jan Algermissen wrote:

 HI,

 ok, so I found token() [1], and that it is an option for paging through
 randomly partitioned data.

 I take it that combining token() and LIMIT is the CQL3 idiom for paging
 (set aside the fact that one shouldn't raelly want to page and use C*)

 Now, when I page through a CF with wide rows, limitting each 'page' to,
 for example, 100 I end up in situations where not all 'sub'rows that have
 the same result for token() are returned because LIMIT chops off the result
 after 100 'sub'rows, not neccessarily at the boundary to the next wide row.

 Obvious ... but inconvenient.

 The solution would be to throw away the last token returned (because
 it's wide row could have been chopped off) and do the next query with the
 token before.

 So instead of doing

   SELECT * FROM users WHERE token(name)  
 token(last-name-of-prev-**result)
 LIMIT 100;

 I'd be doing

  SELECT * FROM users WHERE token(name) 
 token(one-befoe-the-last-name-**of-prev-result) LIMIT 100;


 Question: Is that what I have to do or is there a way to make token()
 and limit work together to return complete wide rows?


 Jan



 [1] token() and how it relates to paging is actually quite hard to grasp
 from the docs.






Re: Vnodes, adding a node ?

2013-08-14 Thread Richard Low
On 14 August 2013 20:02, Andrew Cobley a.e.cob...@dundee.ac.uk wrote:

 I have  small test cluster of 2 nodes.  I ran a stress test on it and with
 nodetool status received the following:

 /usr/local/bin/apache-cassandra-2.0.0-rc1/log $ ../bin/nodetool status
 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  Address   Load   Tokens  Owns (effective)  Host ID
   Rack
 UN  192.168.0.11  141.13 MB  256 49.2%
 4d281e2e-efd9-4abf-bb70-ebdf8e2b4fc3  rack1
 UN  192.168.0.10  145.59 MB  256 50.8%
 7fc5795a-bd1b-4e42-88d6-024c5216a893  rack1

 I then added a third node with no machines writing to the system.  Using
 nodetool status I got the following:

 /usr/local/bin/apache-cassandra-2.0.0-rc1/log $ ../bin/nodetool status
 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  Address   Load   Tokens  Owns (effective)  Host ID
   Rack
 UN  192.168.0.11  141.12 MB  256 32.2%
 4d281e2e-efd9-4abf-bb70-ebdf8e2b4fc3  rack1
 UN  192.168.0.10  145.59 MB  256 35.3%
 7fc5795a-bd1b-4e42-88d6-024c5216a893  rack1
 UN  192.168.0.12  111.9 KB   256 32.5%
 e5e6d8bd-c652-4c18-8fa3-3d71471eee65  rack1

 Is this correct ?  I was under the impression that adding a node to an
 existing cluster would distribute the load around the cluster. Am I perhaps
 missing a step or have a config error perhaps ?


How did you add the node?  It looks like it didn't bootstrap but just
joined the ring.  You need to make sure the node is not set as a seed and
that auto_bootstrap is true (the default).

Alternatively, you could run 'nodetool rebuild' to stream data from the
other nodes.

Richard.


Re: cassandra 1.2.5- virtual nodes (num_token) pros/cons?

2013-08-13 Thread Richard Low
On 13 August 2013 10:15, Alain RODRIGUEZ arodr...@gmail.com wrote:

Streaming from all the physical nodes in the cluster should make repair
 faster, for the same reason it makes bootstrap faster. Shouldn't it ?


Virtual nodes doesn't speed up either very much.  Repair and bootstrap will
be limited by the node doing repair or bootstrap, since it has to do the
same amount of work whatever num_tokens is.  It places a more even load
across the rest of the cluster though, since it will repair with or
bootstrap from all nodes in the cluster.  So the overall time will in most
cases be about the same.

The real speedup from vnodes comes when running removenode, when the
streaming happens in parallel across all nodes.

Richard.


Re: cassandra 1.2.5- virtual nodes (num_token) pros/cons?

2013-08-06 Thread Richard Low
On 6 August 2013 08:40, Aaron Morton aa...@thelastpickle.com wrote:

 The reason for me looking at virtual nodes is because of terrible
 experiences we had with 0.8 repairs and as per documentation (an logically)
 the virtual nodes seems like it will help repairs being smoother. Is this
 true?

 I've not thought too much about how they help repair run smoother, what
 was the documentation you read ?


There might be a slight improvement but I haven't observed any.  The
difference might be that, because every node shares replicas with every
other (with high probability), a single repair operation does the same work
on the node it was called on, but the rest is spread out over the cluster,
rather than just the RF nodes either side of the repairing node.  This
means the post-repair compaction work will take less time and the length of
time a node is loaded for during repair is less.

However, the other benefits of vnodes are likely to be much more useful.

Richard.


Re: clarification of token() in CQL3

2013-08-06 Thread Richard Low
On 6 August 2013 15:12, Keith Freeman 8fo...@gmail.com wrote:

  I've seen in several places the advice to use queries like to this page
 through lots of rows:


 select id from mytable where token(id)  token(last_id)


 But it's hard to find detailed information about how this works (at least
 that I can understand -- the description in the Cassandra manual is pretty
 brief).

 One thing I'd like to know is if new rows are always guaranteed to have
 token(new_id)  token(ids-of-all-previous-rows)?  E.g. if I have one
 process that adds rows to a table, and another that processes rows from the
 table, can the processor save the id of the last row processed and when
 he wakes up use:

 select * from mytable where token(id)  token(last_processed_id)


 to process only new rows?  Will this always work to get only new rows?


No, unfortunately not.  The tokens are generated by the partitioner - they
are the hash of the row key.  New tokens could be anywhere in the range of
tokens so you can't use token ordering to find new rows.

The query you suggest works to page through all the data in your column
family.  Rows will be returned regardless of when they were added (as long
as they were added before the query started).  Finding rows that have been
added since a certain time is hard in Cassandra since they are stored in
token order.  In general you have to read through all the data and work out
from e.g. a date field if they should be treated as new.

Richard.


Re: clarification of token() in CQL3

2013-08-06 Thread Richard Low
On 6 August 2013 16:56, Keith Freeman 8fo...@gmail.com wrote:

 Your description makes me think that if new rows are added during the
 paging (i.e. between one select with token()'s and another), they might
 show up in the query results, right?  (because the hash of the new row keys
 might fall sequentially after token(last_processed_row))


Yes, new rows will appear if their hash is greater than last_processed_row.

Richard.


Re: Reducing the number of vnodes

2013-08-05 Thread Richard Low
On 5 August 2013 12:30, Christopher Wirt chris.w...@struq.com wrote:

I’m thinking about reducing the number of vnodes per server.

 ** **

 We have 3 DC setup – one with 9 nodes, two with 3 nodes each.

 ** **

 Each node has 256 vnodes. We’ve found that repair operations are beginning
 to take too long.

 ** **

 Is reducing the number of vnodes to 64/32 likely to help our situation?


Unlikely.  The amount of time repair takes only depends on the number of
vnodes if you have a tiny amount of data.  If you have 256 vnodes and not
much more than 256 * num_nodes rows, the overhead of splitting up the
repairs into separate ranges for each vnode is significant.  However, once
your dataset becomes bigger than this trivial amount, the vnode overhead of
repair becomes totally insignificant.

The main reasons for slow repair are lots of data or lots of data out of
sync.  You can tell how much is out of sync by looking in the logs - it
will say how many ranges within each vnode range need repairing.

Richard.


Re: Counters and replication

2013-08-05 Thread Richard Low
On 5 August 2013 20:04, Christopher Wirt chris.w...@struq.com wrote:

 Hello,

 ** **

 Question about counters, replication and the ReplicateOnWriteStage

 ** **

 I’ve recently turned on a new CF which uses a counter column. 

 ** **

 We have a three DC setup running Cassandra 1.2.4 with vNodes, hex core
 processors, 32Gb memory.

 DC 1 - 9 nodes with RF 3

 DC 2 - 3 nodes with RF 2 

 DC 3 - 3 nodes with RF 2

 ** **

 DC 1 one receives most of the updates to this counter column. ~3k per sec.
 

 ** **

 I’ve disabled any client reads while I sort out this issue.

 Disk utilization is very low

 Memory is aplenty (while not reading)

 Schema:

 CREATE TABLE cf1 (

   uid uuid,

   id1 int,

   id2 int,

   id3 int,

   ct counter,

   PRIMARY KEY (uid, id1, id2, id3)

 ) WITH …

 ** **

 Three of the machines in DC 1 are reporting very high CPU load.

 Looking at tpstats there is a large number of pending
 ReplicateOnWriteStage just on those machines.

 ** **

 Why would only three of the machines be reporting this? 

 Assuming its distributed by uuid value there should be an even load across
 the cluster, yea?

 Am I missing something about how distributed counters work?


If you have many different uid values and your cluster is balanced then you
should see even load.  Were your tokens chosen randomly?  Did you start out
with num_tokens set high or upgrade from num_tokens=1 or an earlier
Cassandra version?  Is it possible your workload is incrementing the
counter for one particular uid much more than the others?

The distribution of counters works the same as for non-counters in terms of
which nodes receive which values.  However, there is a read on the
coordinator (randomly chosen for each inc) to read the current value and
replicate it to the remaining replicas.  This makes counter increments much
more expensive than normal inserts, even if all your counters fit in cache.
 This is done in the ReplicateOnWriteStage, which is why you are seeing
that queue build up.


 **

 Is changing CL to ONE fine if I’m not too worried about 100% consistency?


Yes, but to make the biggest difference you will need to turn off
replicate_on_write (alter table cf1 with replicate_on_write = false;) but
this *guarantees* your counts aren't replicated, even if all replicas are
up.  It avoids doing the read, so makes a huge difference to performance,
but means that if a node is unavailable later on, you *will* read
inconsistent counts.  (Or, worse, if a node fails, you will lose counts
forever.)  This is in contrast to CL.ONE inserts for normal values when
inserts are still attempted on all replicas, but only one is required to
succeed.

So you might be able to get a temporary performance boost by changing
replicate_on_write if your counter values aren't important.  But this won't
solve the root of the problem.

Richard.


Re: Installing Debian package from ASF repo

2013-07-29 Thread Richard Low
On 29 July 2013 12:00, Pavel Kirienko pavel.kirienko.l...@gmail.com wrote:

 Hi,

 I failed to install the Debian package of Cassandra 1.2.7 from ASF
 repository because of 404 error.
 APT said:

 http://www.apache.org/dist/cassandra/debian/pool/main/c/cassandra/cassandra_1.2.7_all.deb
  404  Not Found [IP: 192.87.106.229 80]

 http://www.apache.org/dist/cassandra/debian/pool/main/c/cassandra/cassandra_1.2.7_all.deb
  404  Not Found [IP: 140.211.11.131 80]

 The directory listing shows that there is no cassandra_1.2.7_all.deb
 here: http://www.apache.org/dist/cassandra/debian/pool/main/c/cassandra/

 Seems that I need to install it from .deb by hand, right? When 1.2.7 will
 be available in the ASF or DataStax repository?


1.2.7 has been removed due to a regression - see the 1.2.8 release
announcement.  However, 1.2.8 debs have to be installed manually until
Sylvain is back.  You can find them at http://people.apache.org/~eevans/.

Richard.


Re: nodetool cfstats write count ?

2013-07-29 Thread Richard Low
On 29 July 2013 14:43, Langston, Jim jim.langs...@compuware.com wrote:

 Running nodetool and looking at the cfstats output, for the
 counters such as write count and read count, do those numbers
 reflect any replication ?


  For instance, if write count shows 3000 and the replication factor
 is 3, is that really 1000 writes ?


The counts are the number of operations that particular node has processed.
 So if you sum up all the write counts across all node's cfstats output, it
will be a factor of replication factor too high, assuming all replicas
received all writes.  (A write message will be sent to all replicas, but if
one is down or busy it may not get processed.)

If your cluster is balanced, then you can estimate the total number of
write operations from one cfstats output as

count * num_nodes / replication_factor

So, as in your example, if one node shows write count 3000 and you have RF
3 with e.g. 3 nodes, your cluster will have processed about 3000 writes.

Reads are a bit harder, because, unlike writes, not all nodes necessarily
receive all read requests.  It depends on consistency level, snitch and
value of read_repair_chance.  You can estimate how many nodes will be
involved in each read request though to figure out how many reads have
actually been submitted.

However, I think the numbers exposed by
StorageProxyMBean getWriteOperations() and getReadOperations() would give
you the true number (when summed up over all nodes).  These numbers don't
over count for replication since they count client requests.  I don't think
these are exposed by any nodetool commands though, but you can use any JMX
client to read them.

Richard.


Re: Cassandra and RAIDs

2013-07-24 Thread Richard Low
On 24 July 2013 15:36, Jan Algermissen jan.algermis...@nordsc.com wrote:


 is it recommended to set up Cassandra using 'RAID-ed' disks for per-node
 reliability or do people usually just rely on having the multiple nodes
 anyway - why bother with replicated disks?


It's not necessary, due to replication as you say.  You can give Cassandra
your JBOD disks and it will split data between them and avoid a disk (or
fail the node, you can choose) if one fails.

There are some reasons to consider RAID though:

* It is probably quicker and places no load on the rest of the cluster to
do a RAID rebuild rather than a nodetool rebuild/repaid.  The importance of
this depends on how much data you have and the load on your cluster.  If
you don't have much data per node or if there is spare capacity then RAID
will offer no benefit here.
* Using JBOD, the largest SSTable you can have is limited to the size of
one disk.  This is unlikely to cause problems in most scenarios but an
erroneous nodetool compact could cause problems if your data size is
greater than can fit on any one disk.

Richard.


Re: [deletion in the future]

2013-07-20 Thread Richard Low
On 19 July 2013 23:31, Alexis Rodríguez arodrig...@inconcertcc.com wrote:

 Hi guys,

 I've read here [1] that you can make a deletion mutation for the future.
 That mechanism operates as a schedule for deletions according to the
 stackoverflow post. But, I've been having problems to make it work with
 my thrift c++ client. I believe it's related to this paragraph of the
 thrift api documentation:


What problem are you having?  If you insert a tombstone with a future
timestamp, it means delete what is there now, as well as anything that is
inserted in the future up to the tombstone timestamp.  It does not mean for
the tombstone to only kick in in the future, that is what TTLs do.

Richard.


Re: [deletion in the future]

2013-07-20 Thread Richard Low
On 20 July 2013 15:16, Alexis Rodríguez arodrig...@inconcertcc.com wrote:

 That's exactly what is happening with my row, but not what I was trying to
 do. It seems that I misunderstood the stackoverflow post. I was trying to
 schedule a delete for an entire row, is using ttl for columns the only way?


Yes, there's no TTL for rows.

Richard.


Re: Cassandra with vnode and ByteOrderedPartition

2013-07-03 Thread Richard Low
On 3 July 2013 21:04, Sávio Teles savio.te...@lupa.inf.ufg.br wrote:

We're using ByteOrderedPartition to programmatically choose the machine
 which a objet will be inserted.*

 *How can I use *ByteOrderedPartition *with vnode on Cassandra 1.2?


Don't.  Managing tokens with ByteOrderedPartitioner is very hard anyway,
but with vnodes you have to manually manage many more tokens.  Also I doubt
BOP + vnodes has had any production testing.

Richard.


Re: Cassandra with vnode and ByteOrderedPartition

2013-07-03 Thread Richard Low
On 3 July 2013 22:18, Sávio Teles savio.te...@lupa.inf.ufg.br wrote:

 We were able to implement ByteOrderedPartition on Cassandra 1.1 and
 insert an object in a specific machine.

 However, with Cassandra 1.2 and VNodes we can't implement VNode with 
 ByteOrderedPartitioner
 to insert an object in a specific machine.


You don't have to use vnodes in Cassandra 1.2 - set num_tokens to 1 to
disable.

Richard.


Re: [Cassandra] Expanding a Cassandra cluster

2013-06-18 Thread Richard Low
On 10 June 2013 22:00, Emalayan Vairavanathan svemala...@yahoo.com wrote:

b) Will Cassandra automatically take care of removing
 obsolete keys in future ?


In a future version Cassandra should automatically clean up for you:

https://issues.apache.org/jira/browse/CASSANDRA-5051

Right now though you have to run cleanup eventually or the space will never
be reclaimed.

Richard.


Re: Why so many vnodes?

2013-06-11 Thread Richard Low
On 11 June 2013 09:54, Theo Hultberg t...@iconara.net wrote:

But in the paragraph just before Richard said that finding the node that
 owns a token becomes slower on large clusters with lots of token ranges, so
 increasing it further seems contradictory.


I do mean increase for larger clusters, but I guess it depends on what you
are optimizing for.  If you care about maintaining an even load, where
differences are measured relative to the amount of data each node has, then
you need T  N.

However, you're right, this can slow down some operations.  Repair has a
fixed cost for each token so gets a bit slower with higher T.  Finding
which node owns a range gets harder with T but this code was optimized so I
don't think it will become a practical issue.

Is this a correct interpretation: finding the node that owns a particular
 token becomes slower as the number of nodes (and therefore total token
 ranges) increases, but for large clusters you also need to take the time
 for bootstraps into account, which will become slower if each node has
 fewer token ranges. The speed referred to in the two cases are the speeds
 of different operations, and there will be a trade off, and 256 initial
 tokens is a trade off that works for most cases.


Yes this is right.  The bootstraps may become slower because the node is
streaming from fewer original nodes (although it may only show on very busy
clusters, since otherwise bootstrap is limited by the joining node).  But
more importantly I think is that new nodes won't take an even share of the
data if T is too small.

Richard.


Re: Why so many vnodes?

2013-06-10 Thread Richard Low
Hi Theo,

The number (let's call it T and the number of nodes N) 256 was chosen to
give good load balancing for random token assignments for most cluster
sizes.  For small T, a random choice of initial tokens will in most cases
give a poor distribution of data.  The larger T is, the closer to uniform
the distribution will be, with increasing probability.

Also, for small T, when a new node is added, it won't have many ranges to
split so won't be able to take an even slice of the data.

For this reason T should be large.  But if it is too large, there are too
many slices to keep track of as you say.  The function to find which keys
live where becomes more expensive and operations that deal with individual
vnodes e.g. repair become slow.  (An extreme example is SELECT * LIMIT 1,
which when there is no data has to scan each vnode in turn in search of a
single row.  This is O(NT) and for even quite small T takes seconds to
complete.)

So 256 was chosen to be a reasonable balance.  I don't think most users
will find it too slow; users with extremely large clusters may need to
increase it.

Richard.


On 10 June 2013 18:55, Theo Hultberg t...@iconara.net wrote:

 I'm not sure I follow what you mean, or if I've misunderstood what
 Cassandra is telling me. Each node has 256 vnodes (or tokens, as the
 prefered name seems to be). When I run `nodetool status` each node is
 reported as having 256 vnodes, regardless of how many nodes are in the
 cluster. A single node cluster has 256 vnodes on the single node, a six
 node cluster has 256 nodes on each machine, making 1590 vnodes in total.
 When I run `SELECT tokens FROM system.peers` or `nodetool ring` each node
 lists 256 tokens.

 This is different from how it works in Riak and Voldemort, if I'm not
 mistaken, and that is the source of my confusion.

 T#


 On Mon, Jun 10, 2013 at 4:54 PM, Milind Parikh milindpar...@gmail.comwrote:

 There are n vnodes regardless of the size of the physical cluster.
 Regards
 Milind
 On Jun 10, 2013 7:48 AM, Theo Hultberg t...@iconara.net wrote:

 Hi,

 The default number of vnodes is 256, is there any significance in this
 number? Since Cassandra's vnodes don't work like for example Riak's, where
 there is a fixed number of vnodes distributed evenly over the nodes, why so
 many? Even with a moderately sized cluster you get thousands of slices.
 Does this matter? If your cluster grows to over thirty machines and you
 start looking at ten thousand slices, would that be a problem? I guess trat
 traversing a list of a thousand or ten thousand slices to find where a
 token lives isn't a huge problem, but are there any other up or downsides
 to having a small or large number of vnodes per node?

 I understand the benefits for splitting up the ring into pieces, for
 example to be able to stream data from more nodes when bootstrapping a new
 one, but that works even if each node only has say 32 vnodes (unless your
 cluster is truly huge).

 yours,
 Theo





Re: cassandra-shuffle time to completion and required disk space

2013-05-01 Thread Richard Low
Hi John,

 - Each machine needed enough free diskspace to potentially hold the
entire cluster's sstables on disk

I wrote a possible explanation for why Cassandra is trying to use too much
space on your ticket:

https://issues.apache.org/jira/browse/CASSANDRA-5525

if you could provide the information there we can hopefully fix it.

Richard.


Re: Problems with shuffle

2013-04-15 Thread Richard Low
On 14 April 2013 00:56, Rustam Aliyev rustam.li...@code.az wrote:

  Just a followup on this issue. Due to the cost of shuffle, we decided
 not to do it. Recently, we added new node and ended up in not well balanced
 cluster:

 Datacenter: datacenter1
 ===
 Status=Up/Down
 |/ State=Normal/Leaving/Joining/Moving
 --  Address   Load   Tokens  Owns   Host
 ID   Rack
 UN  10.0.1.8  52.28 GB   260 18.3%
 d28df6a6-c888-4658-9be1-f9e286368dce  rack1
 UN  10.0.1.11 55.21 GB   256 9.4%
 7b0cf3c8-0c42-4443-9b0c-68f794299443  rack1
 UN  10.0.1.2  49.03 GB   259 17.9%
 2d308bc3-1fd7-4fa4-b33f-cbbbdc557b2f  rack1
 UN  10.0.1.4  48.51 GB   255 18.4%
 c253dcdf-3e93-495c-baf1-e4d2a033bce3  rack1
 UN  10.0.1.1  67.14 GB   253 17.9%
 4f77fd70-b134-486b-9c25-cfea96b6d412  rack1
 UN  10.0.1.3  47.65 GB   253 18.0%
 4d03690d-5363-42c1-85c2-5084596e09fc  rack1

 It looks like new node took from each other node equal amount of vnodes -
 which is good. However, it's not clear why it decided to have twice less
 than other nodes.


I think this is expected behaviour when adding a node to a cluster that has
been upgraded to vnodes without shuffling.  The old nodes have equally
spaced contiguous tokens.  The new node will choose 256 random new tokens,
which will on average bisect the old ranges.  This means each token the new
node has will only cover half the range (on average) as the old ones.

However, the thing that really matters is the load, which is surprisingly
balanced at 55 GB.  This isn't guaranteed though - it could be about half
or it could be significantly more.  The problem with not doing the shuffle
is the vnode after all the contiguous vnodes for a certain node will be the
target for the second replica of *all* the vnodes for that node.  E.g. if
node A has tokens 10, 20, 30, 40, node B has tokens 50, 60, 70, 80 and node
C (the new node) chooses token 45, it will store a replica for all data
stored in A's tokens.  This is exactly the same reason as why tokens in a
multi-DC deployment need to be interleaved rather than be contiguous.

If shuffle isn't going to work, you could instead decommission each node
then bootstrap it in again.  In principle that should copy your data twice
as much as required (shuffle is optimal in terms of data transfer) but some
implementation details might make it more efficient.

Richard.


Re: Virtual Nodes, lots of physical nodes and potentially increasing outage count?

2012-12-11 Thread Richard Low
Hi Eric,

The time to recover one node is limited by that node, but the time to
recover that's most important is just the time to replicate the data that
is missing from that node.  This is the removetoken operation (called
removenode in 1.2), and this gets faster the more nodes you have.

Richard.


On 11 December 2012 08:39, Eric Parusel ericparu...@gmail.com wrote:

 Thanks for your thoughts guys.

 I agree that with vnodes total downtime is lessened.  Although it also
 seems that the total number of outages (however small) would be greater.

 But I think downtime is only lessened up to a certain cluster size.

 I'm thinking that as the cluster continues to grow:
   - node rebuild time will max out (a single node only has so much write
 bandwidth)
   - the probability of 2 nodes being down at any given time will continue
 to increase -- even if you consider only non-correlated failures.

 Therefore, when adding nodes beyond the point where node rebuild time
 maxes out, both the total number of outages *and* overall downtime would
 increase?

 Thanks,
 Eric




 On Mon, Dec 10, 2012 at 7:00 AM, Edward Capriolo edlinuxg...@gmail.comwrote:

 Assuming you need to work with quorum in a non-vnode scenario. That means
 that if 2 nodes in a row in the ring are down some number of quorum
 operations will fail with UnavailableException (TimeoutException right
 after the failures). This is because the for a given range of tokens quorum
 will be impossible, but quorum will be possible for others.

 In a vnode world if any two nodes are down,  then the intersection of
 vnode token ranges they have are unavailable.

 I think it is two sides of the same coin.


 On Mon, Dec 10, 2012 at 7:41 AM, Richard Low r...@acunu.com wrote:

 Hi Tyler,

 You're right, the math does assume independence which is unlikely to be
 accurate.  But if you do have correlated failure modes e.g. same power,
 racks, DC, etc. then you can still use Cassandra's rack-aware or DC-aware
 features to ensure replicas are spread around so your cluster can survive
 the correlated failure mode.  So I would expect vnodes to improve uptime in
 all scenarios, but haven't done the math to prove it.

 Richard.






-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Vnode migration path

2012-12-11 Thread Richard Low
Hi Mike,

There's also the shuffle utility (in the bin directory) that can
incrementally move ranges around to migrate to vnodes.

Richard.


On 11 December 2012 08:47, Michael Kjellman mkjell...@barracuda.com wrote:

 So I'm wondering if anyone has given thought to their migration path to
 Vnodes. Other than having a separate cluster and migrating the data from
 the old cluster to the vnode cluster what else can we do.

 One suggestion I've heard is start up a second Cassandra instance on each
 node on different ports and migrate between nodes that way.

 Best,
 mike

 'Like' us on Facebook for exclusive content and other resources on all
 Barracuda Networks solutions.

 Visit http://barracudanetworks.com/facebook







-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Why Secondary indexes is so slowly by my test?

2012-12-11 Thread Richard Low
Hi,

Secondary index lookups are more complicated than normal queries so will be
slower.  Items have to first be queried in the index, then retrieved from
their actual location.  Also, inserting into indexed CFs will be slower
(but will get substantially faster in 1.2 due to CASSANDRA-2897).

If you need to retrieve large amounts of data with your query, you would be
better off changing your data model to not use secondary indexes.

Richard.


On 7 December 2012 03:08, Chengying Fang cyf...@ngnsoft.com wrote:

 Hi guys,

 I found Secondary indexes too slowly in my product(amazon large instance)
 with cassandra, then I did test again as describe here. But the result is
 the same as product. What's wrong with cassandra or me?
 Now my test:
 newly installed ubuntu-12.04 LTS , apache-cassandra-1.1.6, default
 configure, just one keyspace(test) and one CF(TestIndex):

1. CREATE COLUMN FAMILY TestIndex
2. WITH comparator = UTF8Type
3. AND key_validation_class=UTF8Type
4. AND default_validation_class = UTF8Type
5. AND column_metadata = [
6.
{column_name: tk, validation_class: UTF8Type, index_type: KEYS}
7. {column_name: from, validation_class: UTF8Type}
8. {column_name: to, validation_class: UTF8Type}
9. {column_name: tm, validation_class: UTF8Type}
10. ];

 and 'tk' just three value:'A'(1000row),'B'(1000row),'X'(increment by test)
 The test query from cql:
 1,without index:select count(*) from TestIndex limit 100;
 2,with index:select count(*) from TestIndex where tk='X' limit 100;
 When I insert 6 row 'X', the time:1s and 12s.
 When 'X'  up to 13,the time:2.3s and 33s.
 When 'X'  up to 25,the time:3.8s and 53s.

 According to this, when 'X' up to billon, what's the result? Can Secondary
 indexes be used in product? I hope it's my mistake in doing this test.Can
 anyone give some tips about it?
 Thanks in advance.
 fancy




-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Virtual Nodes, lots of physical nodes and potentially increasing outage count?

2012-12-10 Thread Richard Low
Hi Tyler,

You're right, the math does assume independence which is unlikely to be
accurate.  But if you do have correlated failure modes e.g. same power,
racks, DC, etc. then you can still use Cassandra's rack-aware or DC-aware
features to ensure replicas are spread around so your cluster can survive
the correlated failure mode.  So I would expect vnodes to improve uptime in
all scenarios, but haven't done the math to prove it.

Richard.


On 9 December 2012 17:50, Tyler Hobbs ty...@datastax.com wrote:

 Nicolas,

 Strictly speaking, your math makes the assumption that the failure of
 different nodes are probabilistically independent events. This is, of
 course, not a accurate assumption for real world conditions.  Nodes share
 racks, networking equipment, power, availability zones, data centers, etc.
 So, I think the mathematical assertion is not quite as strong as one would
 like, but it's certainly a good argument for handling certain types of node
 failures.


 On Fri, Dec 7, 2012 at 11:27 AM, Nicolas Favre-Felix nico...@acunu.comwrote:

 Hi Eric,

 Your concerns are perfectly valid.

 We (Acunu) led the design and implementation of this feature and spent a
 long time looking at the impact of such a large change.
 We summarized some of our notes and wrote about the impact of virtual
 nodes on cluster uptime a few months back:
 http://www.acunu.com/2/post/2012/10/improving-cassandras-uptime-with-virtual-nodes.html
 .
 The main argument in this blog post is that you only have a failure to
 perform quorum read/writes if at least RF replicas fail within the time it
 takes to rebuild the first dead node. We show that virtual nodes actually
 decrease the probability of failure, by streaming data from all nodes and
 thereby improving the rebuild time.

 Regards,

 Nicolas


 On Wed, Dec 5, 2012 at 4:45 PM, Eric Parusel ericparu...@gmail.comwrote:

 Hi all,

 I've been wondering about virtual nodes and how cluster uptime might
 change as cluster size increases.

 I understand clusters will benefit from increased reliability due to
 faster rebuild time, but does that hold true for large clusters?

 It seems that since (and correct me if I'm wrong here) every physical
 node will likely share some small amount of data with every other node,
 that as the count of physical nodes in a Cassandra cluster increases (let's
 say into the triple digits) that the probability of at least one failure to
 Quorum read/write occurring in a given time period would *increase*.

 Would this hold true, at least until physical nodes becomes greater than
 num_tokens per node?

 I understand that the window of failure for affected ranges would
 probably be small but we do Quorum reads of many keys, so we'd likely hit
 every virtual range with our queries, even if num_tokens was 256.

 Thanks,
 Eric





 --
 Tyler Hobbs
 DataStax http://datastax.com/




-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Hector counter question

2012-03-20 Thread Richard Low
On 20 March 2012 06:51, Tamar Fraenkel ta...@tok-media.com wrote:

 But the increment is thread safe right? if I have two threads trying to 
 increment a counter, then they won't step on each other toe?

That's right - you can have many threads incrementing the same counter
and it's safe.  But reading and incrementing is unsafe.

Richard.

--
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Doubts related to composite type column names/values

2011-12-20 Thread Richard Low
On Tue, Dec 20, 2011 at 5:28 PM, Ertio Lew ertio...@gmail.com wrote:
 With regard to the composite columns stuff in Cassandra, I have the
 following doubts :

 1. What is the storage overhead of the composite type column names/values,

The values are the same.  For each dimension, there is 3 bytes overhead.

 2. what exactly is the difference between the DynamicComposite and Static
 Composite ?

Static composite type has the types of each dimension specified in the
column family definition, so all names within that column family have
the same type.  Dynamic composite type lets you specify the type for
each column, so they can be different.  There is extra storage
overhead for this and care must be taken to ensure all column names
remain comparable.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: GC for ParNew on 0.8.6

2011-10-07 Thread Richard Low
Hi Philippe,

On Thu, Sep 29, 2011 at 6:47 AM, Philippe watche...@gmail.com wrote:
 No it was an upgrade from 0.8.4 or 0.8.5 depending on the nodes.
 No cassandra-env files were changed during the update.
 Any other ideas?  The cluster has just been weird ever since running 0.8.6 :
 has anyone else upgraded and not run into this?

What do you mean by the cluster has been weird since the upgrade?
Have you noticed slow-downs?  Any other messages in the logs that have
appeared since the upgrade?

Richard.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Cassandra 0.8 Counters Inverted Index?

2011-10-03 Thread Richard Low
On Mon, Oct 3, 2011 at 9:14 AM, Pierre-Yves Ritschard
p...@smallrivers.com wrote:
 Unfortunately there's no way to do this in Cassandra right now, except
 by using another row as index, like you're doing right now.

 Of course you could also store by source_id.date and have a batch job
 iterate over all sources to compute the top 100. It would not be real
 time any more though.

Indexes are used to trade-off some insert performance for write
performance.  The index you describe is optimal for reads, so writes
take a hit.  As Pierre says, the only way to maintain an index in
Cassandra is to read, delete and insert on every increment.  This is
how secondary indexes work under the hood in Cassandra, although they
are not implemented for counters.  It's more expensive for counters
though since a counter read is in general more expensive.

So to speed up inserts, you have to take the hit on reads.  The other
extreme is to not build an index at all and read in all the counters
and sort on the client.  But given you have 10,000s of counters, this
will be slow, but inserts are optimal.  A batch job will work too,
provided you are happy to have it non-real time, or slightly out of
date.

Richard.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: 15 seconds to increment 17k keys?

2011-09-02 Thread Richard Low
On Thu, Sep 1, 2011 at 5:16 PM, Ian Danforth idanfo...@numenta.com wrote:

 Does this scale with multiples of the replication factor or directly
 with number of nodes? Or more succinctly, to double the writes per
 second into the cluster how many more nodes would I need?

The write throughput scales with number of nodes, so double to get
double the write capacity.

Increasing the replication factor in general doesn't improve
performance (and increasing without increasing number of nodes
decreases performance).  This is because operations are performed on
all available replicas (with the exception of reads with low
consistency levels and read_repair_chance  1.0).

Note also that there is just one read per counter increment, not a
read per replica.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: 15 seconds to increment 17k keys?

2011-09-01 Thread Richard Low
Assuming you have replicate_on_write enabled (which you almost
certainly do for counters), you have to do a read on a write for each
increment.  This means counter increments, even if all your data set
fits in cache, are significantly slower than normal column inserts.  I
would say ~1k increments per second is about right, although you can
probably do some tuning to improve this.

I've also found that the pycassa client uses significant amounts of
CPU, so be careful you are not CPU bound on the client.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu

On Thu, Sep 1, 2011 at 2:31 AM, Yang tedd...@gmail.com wrote:
 1ms per add operation is the general order of magnitude I have seen with my
 tests.


 On Wed, Aug 31, 2011 at 6:04 PM, Ian Danforth idanfo...@numenta.com wrote:

 All,

  I've got a 4 node cluster (ec2 m1.large instances, replication = 3)
 that has one primary counter type column family, that has one column
 in the family. There are millions of rows. Each operation consists of
 doing a batch_insert through pycassa, which increments ~17k keys. A
 majority of these keys are new in each batch.

  Each operation is taking up to 15 seconds. For our system this is a
 significant bottleneck.

  Does anyone know if this write speed is expected?

 Thanks in advance,

  Ian




Re: hw requirements

2011-08-29 Thread Richard Low
Hi,

The hardware you choose depends a bit on your workload - writes vs
reads, amount of cacheable data, latency requirements, etc..  What
sort of workload do you expect?

See http://wiki.apache.org/cassandra/CassandraHardware for some general advice.

People typically have 8-24 GB RAM per node with 1-8 TB of storage, but
there are cases where bigger or smaller makes sense.  Don't overspec
your nodes - you'll be better off with more smaller nodes.

You can use SSDs if you need the random read rate, and SATA drives are fine too.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu

On Mon, Aug 29, 2011 at 2:15 PM, Helder Oliveira
helder.olive...@byside.com wrote:
 Hello guys,

 What is the type of profile of a cassandra server.
 Are SSD an option ?
 Does cassandra needs better CPU ou lots of memory ?
 Are SATA II disks ok ?

 I am making some tests, and i started evaluating the possible hardware.

 If someone already has conclusions about it, please share :D

 Thanks a lot.



Pre-CassandraSF Happy Hour on Sunday

2011-07-08 Thread Richard Low
Hi all,

If you're in San Francisco for CassandraSF on Monday 11th, then come
and join fellow Cassandra users and committers on Sunday evening.
Starting at 6:30pm at ThirstyBear, the famous brewing company.  We'll
have drinks, food and more.

RSVP at Eventbrite: http://pre-cassandrasf-happyhour.eventbrite.com/

Hope you can join us!

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: deduct token values for BOP

2011-07-07 Thread Richard Low
On Thu, Jul 7, 2011 at 3:39 PM, A J s5a...@gmail.com wrote:
 Thanks. The above works.
 But when I try to use the binary values rather than the hex values, it
 does not work. i.e. instead of using 64ff, I use 01100100. Instead of
 6Dff, I use 01101101.
 When using the binary values, everything (strings starting with a to
 z) seem to be going to n1 only.
 Any idea why ?

You're writing the binary value as the initial token?  That won't
work, since it expects hex.

Richard.


Re: deduct token values for BOP

2011-07-06 Thread Richard Low
On Wed, Jul 6, 2011 at 3:06 PM, A J s5a...@gmail.com wrote:
 I wish to use the order preserving byte-ordered partitioner. How do I
 figure the initial token values based on the text key value.
 Say I wish to put all keys starting from a to d on N1. e to m on N2
 and n to z on N3. What would be the initial_token values on each of
 the 3 nodes to accomplish this ?

If all keys use characters a-z then the following will work:

N1: 64ff
N2: 6dff
N3: 7aff

(64, 6d and 7a are hex for ascii codes of d, m, z).  Here the key (in
hex) 64 will go to N2 even though it starts with d.  But every
string that starts with a-d with only characters a-z afterwards will
go to N1.

Richard.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: issue with querying SuperColumn

2011-06-21 Thread Richard Low
You have key validation class UTF8Type for the standard CF, but
BytesType for the super.  This is why the key is 1 for standard, but
printed as 31 for super, which is the hex ascii code for 1.  In your
java code, use 1.getBytes() as your key and it should work.

Richard.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu

On Tue, Jun 21, 2011 at 7:36 AM, Vivek Mishra
vivek.mis...@impetus.co.in wrote:




 I am facing one issue with querying superColumn using clien.get() API.
 Although it is working when I try it for a ColumnFamily(rather than
 SuperColumnFamily).



 It is working for:

     ColumnFamily: users

   Key Validation Class: org.apache.cassandra.db.marshal.UTF8Type

   Default column value validator:
 org.apache.cassandra.db.marshal.BytesType

   Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type

   Row cache size / save period in seconds: 0.0/0

   Key cache size / save period in seconds: 20.0/14400

   Memtable thresholds: 0.2953125/63/1440 (millions of ops/MB/minutes)

  GC grace seconds: 864000

   Compaction min/max thresholds: 4/32

   Read repair chance: 1.0

   Replicate on write: false

   Built indexes: []



 Issuing list of users(using Cassandra-cli):



 [default@key1] list users;

 Using default limit of 100

 ---

 RowKey: 1

 = (column=name, value=74657374, timestamp=1308637325517000)





 Java code:

 String key=1;

     ColumnPath columnPath = new ColumnPath(users);

     columnPath.setColumn(name.getBytes());

     ColumnOrSuperColumn colName =
 cassndraClient.get(java.nio.ByteBuffer.wrap(key.getBytes()), columnPath ,
 ConsistencyLevel.ONE);

     Column col = colName.getColumn();

     System.out.println(new String(col.getValue(), UTF-8));





 RESULT: I am getting “test” printed.



 BUT when I tried it for Super column family “SuperCli” :



     ColumnFamily: SuperCli (Super)

   Key Validation Class: org.apache.cassandra.db.marshal.BytesType

   Default column value validator:
 org.apache.cassandra.db.marshal.BytesType

   Columns sorted by:
 org.apache.cassandra.db.marshal.UTF8Type/org.apache.cassandra.db.marshal.UTF8Type

   Row cache size / save period in seconds: 0.0/0

   Key cache size / save period in seconds: 20.0/14400

   Memtable thresholds: 0.2953125/63/1440 (millions of ops/MB/minutes)

   GC grace seconds: 864000

   Compaction min/max thresholds: 4/32

   Read repair chance: 1.0

   Replicate on write: false

   Built indexes: []



 [default@key1] list SuperCli;

 Using default limit of 100

 ---

 RowKey: 31

 = (super_column=address,

  (column=city, value=6e6f696461, timestamp=1308296234977000))

 = (super_column=address1,

  (column=city, value=476e6f696461, timestamp=1308296283221000))

 = (super_column=address2,

  (column=city, value=476e6f696461, timestamp=1308296401951000))



 1 Row Returned.





 Java Code:



 ColumnPath columnPath = new ColumnPath(SuperCli);

     columnPath.setSuper_column(address.getBytes());

     String key=31;

     cassndraClient.get(java.nio.ByteBuffer.wrap(key.getBytes()),
 columnPath , ConsistencyLevel.ONE);





 I am getting exception:



 NotFoundException()

  at
 org.apache.cassandra.thrift.Cassandra$get_result.read(Cassandra.java:6418)

  at
 org.apache.cassandra.thrift.Cassandra$Client.recv_get(Cassandra.java:519)

  at org.apache.cassandra.thrift.Cassandra$Client.get(Cassandra.java:492)

  at CasQuery.main(CasQuery.java:112)









 Any idea about this issue?





 --Vivek

 
 Write to us for a Free Gold Pass to the Cloud Computing Expo, NYC to attend
 a live session by Head of Impetus Labs on ‘Secrets of Building a Cloud
 Vendor Agnostic PetaByte Scale Real-time Secure Web Application on the Cloud
 ‘.

 Looking to leverage the Cloud for your Big Data Strategy ? Attend Impetus
 webinar on May 27 by registering at
 http://www.impetus.com/webinar?eventid=42 .


 NOTE: This message may contain information that is confidential,
 proprietary, privileged or otherwise protected by law. The message is
 intended solely for the named addressee. If received in error, please
 destroy and notify the sender. Any use of this email is prohibited when
 received in error. Impetus does not represent, warrant and/or guarantee,
 that the integrity of this communication has been maintained nor that the
 communication is free of errors, virus, interception or interference.



Re: Retrieving a column from a fat row vs retrieving a single row

2011-06-09 Thread Richard Low
Remember also that partitioning is done by rows, not columns.  So
large rows are stored on a single host.  This means they can't be load
balanced and also all requests to that row will hit one host.  Having
separate rows will allow load balancing of I/Os.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu

On Thu, Jun 9, 2011 at 12:50 AM, aaron morton aa...@thelastpickle.com wrote:
 Just to make things less clear, if you have one row that you are continually
 writing it may end up spread out over several SSTables. Compaction helps
 here to reduce the number of files that must be accessed so long as is can
 keep up. But if you want to read column X and the row is fragmented over 5
 SSTables then each one must be accessed.
  https://issues.apache.org/jira/browse/CASSANDRA-2319  is open to try and
 reduce the number of seeks.
 For now take a look at nodetool cfhistograms to see how many sstables are
 read for your queries.
 Cheers
 -
 Aaron Morton
 Freelance Cassandra Developer
 @aaronmorton
 http://www.thelastpickle.com
 On 9 Jun 2011, at 04:50, Peter Schuller wrote:

 As far as I know, to read a single column cassandra will deserialize a

 bunch of them and then pick the correct one (64KB of data right?)

 Assuming the default setting of 64kb, the average amount deserialized
 given random column access should be 8 kb (not true with row cache,
 but with large rows presumably you don't have row cache).

 Would it be faster to have a row for each id I want to translate? This

 would make keycache less effective, but the amount of data read should

 be smaller.

 It depends on what bottlenecks you're optimizing for. A key is
 expensive in the sense that if (1) increases the size of bloom
 filters for the column family, and it (2) increases the memory cost of
 index sampling, and (3) increases the total data size (typically)
 because the row size is duplicated in both the index and data files.

 The cost of deserialization the same data repeatedly is CPU. So if
 you're nowhere near bottlenecking on disk and the memory trade-off is
 reasonable, it may be a suitable optimization. However, consider that
 unless you're doing order preserving partitioning, accessing those
 rows will be effectively random w.r.t. the locations on disk you're
 reading from so you're adding a lot of overhead in terms of disk I/O
 unless your data set fits comfortably in memory.

 --
 / Peter Schuller




Re: Retrieving a column from a fat row vs retrieving a single row

2011-06-09 Thread Richard Low
2011/6/9 Héctor Izquierdo Seliva izquie...@strands.com:

 Yeah, but if I have RF=3 then there are three nodes that can answer the
 request right?

Yes, if you're happy to read ConsistencyLevel.ONE.


Re: Misc Performance Questions

2011-06-08 Thread Richard Low
Hi AJ,

On Wed, Jun 8, 2011 at 9:29 AM, AJ a...@dude.podzone.net wrote:

 Is there a performance hit when dropping a CF?  What if it contains .5 TB of
 data?  If not, is there a quick and painless way to drop a large amount of
 data w/minimal perf hit?

Dropping a CF is quick - it snapshots the files (which creates hard
links) and removes the CF definition.  To actually delete the data,
remove the snapshot files from your data directory.

 Is there a performance hit running multiple keyspaces on a cluster versus
 only one keyspace given a constant total data size?  Is there some quantity
 limit?

There is a tiny amount of memory used per keyspace, but unless you
have very many keyspaces you won't notice any impact of running
multiple keyspaces.

There is however a difference in running multiple column families
versus putting everything in the same column family and separating
them with e.g. a key prefix.  E.g. if you have a large data set and a
small one, it will be quicker to query the small one if it is in its
own column family.

 Using a Random Partitioner, but with a RF = 1, will the rows still be
 spread-out evenly on the cluster or will there be an affinity to a single
 node (like the one receiving the data from the client)?

The rows will be spread out the same way - RF=1 doesn't affect the
load balancing.

 I see a lot of mention of using RAID-0, but not RAID-5/6.  Why?  Even though
 Cass can tolerate a down node due to data loss, it would still be more
 efficient to just rebuild a bad hdd live, right?

There's a trade-off - RAID-0 will give better performance, but
rebuilds are over a network.  WIth RF  1, RAID-0 is enough so that
that you're unlikely to lose data, but as you say, replacing a failed
node will be slower.

 Maybe perf related:  Will there be a problem having multiple keyspaces on a
 cluster all with different replication factors, from 1-3?

No.

Richard.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu


Re: Misc Performance Questions

2011-06-08 Thread Richard Low
On Wed, Jun 8, 2011 at 12:30 PM, AJ a...@dude.podzone.net wrote:

 There is however a difference in running multiple column families
 versus putting everything in the same column family and separating
 them with e.g. a key prefix.  E.g. if you have a large data set and a
 small one, it will be quicker to query the small one if it is in its
 own column family.


 I assumed that a read would be O(1) for any size CF since Cass is
 implemented with hashmaps.  Do you know why size matters?  (forgive the pun)


You may not notice a difference, but it can happen.

For a query, each SSTable is queried.  If there is more data then
there are (most likely) more SSTables to query, slowing it down.  For
point queries, this isn't so bad because the Bloom filters will help,
but for range queries you will notice a big difference.  You will have
to do more seeks to seek over unwanted data.

It will also help buffer caching to separate them - the small SSTables
are more likely to remain in cache.

-- 
Richard Low
Acunu | http://www.acunu.com | @acunu