Re: Effective allocation of multiple disks

2010-03-12 Thread Ted Zlatanov
On Thu, 11 Mar 2010 12:01:27 -0600 Eric Evans  wrote: 

EE> On Wed, 2010-03-10 at 23:20 -0600, Jonathan Ellis wrote:
>> On Wed, Mar 10, 2010 at 9:31 PM, Anthony Molinaro
>>  wrote:
>> > I would almost recommend just keeping things simple and removing
>> > multiple data directories from the config altogether and just
>> > documenting that you should plan on using OS level mechanisms for
>> > growing diskspace and io.
>> 
>> I think that is a pretty sane suggestion actually. 

EE> Or maybe leave the code as is and just document the situation more
EE> clearly? If you're adding more disks to increase storage capacity
EE> and you don't strictly need the extra IO, then multiple data
EE> directories might be preferable to other forms of aggregation (it's
EE> certainly simpler than say a volume manager).

Could Cassandra use a block device as raw storage?  You avoid the
filesystem overhead and it lets the sysadmin determine the best kind of
device (RAID or not underneath) to allocate.

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 09:35:31 -0800 Ryan King  wrote: 

>> With seed node lists, if I get unlucky I'd be trying to hit a downed
>> node in which case I may as well just use RRDNS and deal with connection
>> failure from the start.

RK> Why would you not deal with connection failure?

I mean it's simpler to deal with one type of connection failure (to any
node in RRDNS) than multiples (to seed node to get node list, then to
random active node from that list).  Sorry if my phrasing was confusing.

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 09:19:28 -0800 Chris Goffinet  wrote: 

CG> At Digg we have automated infrastructure. We use Puppet + our own
CG> in-house system that allows us to query pools of nodes for
CG> 'seeds'. Config files like storage-conf.xml are auto generated on
CG> the fly, and we randomly pick a set of seeds.

CG> Seeds can be per datacenter as well. As soon as a machine is
CG> decommissioned, it no longer gets picked as seed.

On Wed, 3 Mar 2010 11:20:07 -0600 Brandon Williams  wrote: 

BW> 2010/3/3 Ted Zlatanov 
>> My concern is that both RRDNS and seed node lists are vulnerable to
>> individual node failure.

BW> They're not.  That's why they're lists.  If one doesn't work out, move along
BW> to the next.

>> Updating DNS when a node dies means you have
>> to wait until the TTL expires, and if you lower the TTL too much your
>> server will get killed.

BW> Don't do that.  Make your clients keep trying.  Any failure is likely to be
BW> transient anyway, so running around messing with DNS every time a machine is
BW> offline doesn't make much sense.

Thanks for the advice.  I am probably being paranoid about the
connection timeout; we're using Puppet as well so I'll just use it to
generate the seeds portion of the config file *and* a plain list of seed
nodes that each client can retrieve (so they don't have to parse the
XML).

On Wed, 3 Mar 2010 11:22:45 -0600 Jonathan Ellis  wrote: 

JE> We appear to be reaching consensus that this is solving a non-problem,
JE> so I have closed that ticket.

Sure.  Thanks for everyone's opinion, I really appreciate it.

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 12:08:06 -0500 Ian Holsman  wrote: 

IH> We could create a branch or git fork where you guys could develop it,
IH> and if it reaches a usable state and others find it interesting it
IH> could get integrated in then

Thanks, Ian.  Would it be OK to do it as a patch in
http://issues.apache.org/jira/browse/CASSANDRA-846?  Or is there a
reason for using a branch/fork instead?

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 09:04:37 -0800 Ryan King  wrote: 

RK> Something like RRDNS is no more complex that managing a list of seed nodes.

How do your clients at Twitter find server nodes?  Do you just run them
local to each node?

My concern is that both RRDNS and seed node lists are vulnerable to
individual node failure.  Updating DNS when a node dies means you have
to wait until the TTL expires, and if you lower the TTL too much your
server will get killed.

With seed node lists, if I get unlucky I'd be trying to hit a downed
node in which case I may as well just use RRDNS and deal with connection
failure from the start.

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 03 Mar 2010 10:43:19 -0600 Eric Evans  wrote: 

EE> It's entirely possible that you've identified a problem that others
EE> can't see, or haven't yet encountered. I don't see it, but then maybe
EE> I'm just thick.

Getting back to my original question, how do you (and others) find
usable Cassandra nodes from your clients?  It's supposed to be a
decentralized database and yet I only know of centralized ways (RRDNS)
to locate nodes.  Contacting the seeds is not a decentralized solution
and sidesteps the issue.  It also complicates the client unnecessarily.

EE> Either way, if you think this is important, the onus is on you to
EE> demonstrate the merit of your idea and contrib/ or a github project is
EE> one way to do that (the latter has the advantage of not needing to rely
EE> on anyone else).

I'll submit a core patch in a jira ticket.  It's much easier than
writing a full application and IMHO much more useful because it "just
works."  If it gets rejected I'll move to contrib/ as you and Gary
suggested.

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 09:32:33 -0600 Gary Dusbabek  wrote: 

GD> 2010/3/3 Ted Zlatanov :
>> This requires knowledge of the seeds so I need to at least look in
>> storage-conf.xml to find them.  Are you saying there's no chance of
>> Cassandra nodes (or just seeds) announcing themselves, even if it's
>> optional behavior that's off by default?  If so I'll do the contrib mDNS
>> service but it really seems like a backward way to do things.

GD> Nodes already announce themselves, only just to the cluster.  That's
GD> what gossip is for.  I don't see the point of making the announcement
GD> to the subnet at large.

GD> The decision rests with the community.  Obviously, if there is enough
GD> merit to this work, it will find its way into the codebase.  I just
GD> think it falls into the realm of shiny-and-neat (mdns and automatic
GD> discovery is cool) and not in the realm of pragmatic (not reliable
GD> across subnets).

It's currently not possible to find a usable node without running
centralized services like RRDNS or a special mDNS broadcaster as you
suggested.  I don't think this is shiny and neat, it's a matter of
running in a true decentralized environment (which Cassandra is supposed
to fit into).

The subnet limitation is not an issue in my environment (we forward
much, much larger multicast volumes routinely) but I understand routing
multicasts is not everyone's cup of tea.  IMHO it's better than the
current situation and, mDNS being a well-known standard, can at least be
handled at the switch level without code changes.

I can do a patch+ticket for this in the core, making it optional and off
by default, or do the same for a contrib/ service as you suggested.  So
I'd appreciate a +1/-1 quick vote on whether this can go in the core to
save me from rewriting the patch later.

Ted



Re: why have ColumnFamilies?

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 08:56:16 -0600 Jonathan Ellis  wrote: 

JE> I would rather move to a more flexible model ("as many levels of
JE> nesting as you want") than a less-flexible one.

That's very exciting.  I've often wished for "just one more level" while
putting Cassandra schemas together, so I hope this happens in a future
release.

Thanks
Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 08:41:18 -0600 Gary Dusbabek  wrote: 

GD> It wouldn't be a lot work for you to write a mdns service that would
GD> query the seeds for endpoints and publish it to interested clients.
GD> It could go in contrib.

This requires knowledge of the seeds so I need to at least look in
storage-conf.xml to find them.  Are you saying there's no chance of
Cassandra nodes (or just seeds) announcing themselves, even if it's
optional behavior that's off by default?  If so I'll do the contrib mDNS
service but it really seems like a backward way to do things.

Ted



Re: why have ColumnFamilies?

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 15:21:42 +0100 Alexandre Conrad  
wrote: 

AC> 2010/3/3 Ted Zlatanov :
KeySpace-> Row->ColumnFamily->Column[name, value]
>> (a two-level map)
>> 
KeySpace-> Row->SuperColumnFamily->SuperColumn[name]->Column[name, value]
>> (a three-level map)

AC> Thanks for the explanation. So this means that entities under
AC> SuperColumnFamily can only be SuperColumns and ColumnFamilies can only
AC> hold Columns.

AC> I guess the term SuperColumnFamily (which I haven't seen documented)
AC> is used when you want to implicitly say that SuperColumns are used
AC> beneath.

Sorry for the confusion.  I'm using SuperColumnFamily as a shortcut to
say "ColumnFamily of type 'Super'" which is what you'll find in the
configuration.

Your diagram was not incorrect.  It shows both Columns and SuperColumns
under the same family which can't happen in practice but logically do
live under the same branch.  So it's sort of correct but confusing :)

Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 07:57:32 -0600 Gary Dusbabek  wrote: 

GD> 2010/3/3 Ted Zlatanov :
TZ> I need to find Cassandra servers on my network from several types of
TZ> clients and platforms.  The goal is to make adding and removing servers
TZ> painless, assuming a leading window of at least 1 hour.  The discovery
TZ> should be automatic and distributed.  I want to minimize management.

GD> Nothing in the current codebase currently meets these needs.  But then
GD> again, cassandra doesn't need the described functionality.  Zeroconf
GD> confines itself to a single subnet (would require router configuration
GD> to work across subnets so that multicast goes through).  

I looked it up and today, mDNS seems to be the standard name for this
protocol (Bonjour/Rednezvous on Apple).  Zeroconf seems to be the older
name and there's a *lot* of name confusion so I'll just stick to "mDNS."

Here's a decent Java implementation: http://sourceforge.net/projects/jmdns/

I don't think routing multicasts across subnets is a burden.

GD> RRDNS would work, but something would need to keep that updated when
GD> servers go away (it wouldn't be automatic).

GD> If you can count on one of your (seed nodes) to be up, RRDNS could be
GD> used to connect to one of them and fetch the token range list.  To do
GD> this, create a thrift client and call describe_ring.  In older
GD> versions you can get a jsonified endpoint map by calling
GD> get_string_property('token map').

It would really be much more efficient if I didn't have to maintain
RRDNS, but could instead look at the mDNS broadcasts for the Cassandra
service.  What you describe is a centralized model, no?

With mDNS I wouldn't have to know which nodes are up or down, and I
wouldn't have to do extra queries, it would just work.  I don't see why
Cassandra doesn't need that functionality.  How else could you be
guaranteed to find a live node if there is one on your subnet?

Ted



Re: why have ColumnFamilies?

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 14:43:14 +0100 Alexandre Conrad  
wrote: 

AC> 2010/3/3 Ted Zlatanov :
>> I don't understand the advantages of ColumnFamilies over a
>> SuperColumnFamily with just one supercolumn.  Why have the former if the
>> latter is functionally equivalent?

AC> As far as I understand, there's how I organize Cassandra entities:

AC> http://paste.pocoo.org/show/185126/

AC> Is this somehow correct?

This was your diagram (fixed-width font required):

> KeySpace
>|
>+-- Row  
> |   
> +-- ColumnFamily
>  |  
>  +-- Column 
>  |  
>  +-- SuperColumn
>  |  
>  +-- Column 

That's incorrect.  Here's a (shorter, correct) version:

KeySpace->Row->ColumnFamily->Column[name, value]

(a two-level map)

KeySpace->Row->SuperColumnFamily->SuperColumn[name]->Column[name, value]

(a three-level map)

My point was that conceptually, the three-level map can express the
two-level map.

Ted



Re: why have ColumnFamilies?

2010-03-03 Thread Ted Zlatanov
On Wed, 3 Mar 2010 07:23:48 -0600 Jonathan Ellis  wrote: 

JE> 2010/3/3 Ted Zlatanov :
>> I don't understand the advantages of ColumnFamilies over a
>> SuperColumnFamily with just one supercolumn.  Why have the former if the
>> latter is functionally equivalent?

JE> http://issues.apache.org/jira/browse/CASSANDRA-598

So is there a vague plan to move to just SuperColumnFamilies once this
is resolved?  The API can emulate ColumnFamilies, obviously.  Or are
ColumnFamilies here to stay?

Ted



why have ColumnFamilies?

2010-03-03 Thread Ted Zlatanov
I don't understand the advantages of ColumnFamilies over a
SuperColumnFamily with just one supercolumn.  Why have the former if the
latter is functionally equivalent?

Thanks
Ted



Re: finding Cassandra servers

2010-03-03 Thread Ted Zlatanov
On Mon, 01 Mar 2010 12:15:11 -0600 Ted Zlatanov  wrote: 

TZ> I need to find Cassandra servers on my network from several types of
TZ> clients and platforms.  The goal is to make adding and removing servers
TZ> painless, assuming a leading window of at least 1 hour.  The discovery
TZ> should be automatic and distributed.  I want to minimize management.

TZ> Round-robin DNS with a 1-hour TTL would work all right, but I was
TZ> wondering if Bonjour/Zeroconf is a better idea and what else should I
TZ> consider.

So... is this a dumb question or is there no good answer currently to
discovering Cassandra servers?

Ted



finding Cassandra servers

2010-03-01 Thread Ted Zlatanov
I need to find Cassandra servers on my network from several types of
clients and platforms.  The goal is to make adding and removing servers
painless, assuming a leading window of at least 1 hour.  The discovery
should be automatic and distributed.  I want to minimize management.

Round-robin DNS with a 1-hour TTL would work all right, but I was
wondering if Bonjour/Zeroconf is a better idea and what else should I
consider.

Thanks
Ted



Re: cassandra freezes

2010-02-25 Thread Ted Zlatanov
On Thu, 25 Feb 2010 08:56:25 -0600 Jonathan Ellis  wrote: 

JE> Are you swapping?
JE> http://spyced.blogspot.com/2010/01/linux-performance-basics.html

JE> otherwise there's something wrong w/ your vm (?), disk i/o doesn't
JE> block incoming writes in cassandra

If the user has enough memory, can tmpfs (/tmp for example) be used for
the data and commitlog to produce results without disk I/O (so it can be
determined if disk I/O is the problem)?  I've done this with other
applications but don't know if it would work with Cassandra.

Ted



Re: full text search

2010-02-25 Thread Ted Zlatanov
On Wed, 24 Feb 2010 15:41:07 -0800 Mohammad Abed  
wrote: 

MA> Any suggestions on how to pursue full text search with Cassandra, what
MA> options are out there?

I've proposed a bitmask patch
(https://issues.apache.org/jira/browse/CASSANDRA-764) which would help
if your search word set is finite and can be expressed in a bitmask (as
part of the SliceRange).  100,000 tags for instance can be expressed
uncompressed in 100,000 bits which is just a 12.5KB query.  If you use
inversion lists to compress the search string further you can make the
query really tiny, a few bytes per specified tag for even huge search
spaces, but my patch doesn't do that yet.  If this is a viable option
for you, vote for the issue.

I'm using a similar approach on the client side, filtering the results
after I get them, but my search space is IP addresses and I'm filtering
on netmasks so the netmask itself is the bitmask filter.

Expressing the full text search as a supercolumn bitmasked index really
depends on your search domain.  See for instance
http://en.wikipedia.org/wiki/Knowledge_organization but the best place
to start cataloguing general information is to ask a librarian (I say
this having worked at a large search engine that employed dozens of them
to classify the web).

Ted



Re: Row with many columns

2010-02-19 Thread Ted Zlatanov
On Fri, 19 Feb 2010 11:46:25 +0100 Wojciech Kaczmarek  
wrote: 

WK> You can use get_range_slice with the start_key and finish_key crafted like
WK> this:
WK> assume id is "<10-digits-worldId:whatever_objectId>"
WK> to get objects from world with id=42 you do:
WK> start_key='42:', end_key='42~'
WK> (for the ending mark of end_key choose any character which is greater than
WK> your delimiter ':' in a sort order)
WK> Just remember the keys ordering is defined by the specific Partitioner class
WK> used)

It's still not clear what's in each object supercolumn in your current
schema.  How do you encode a relationship between two objects?  How do
you encode an object's properties?  That's the heart of your design.

For the row keys I agree with Wojciech, you need to let Cassandra handle
objects as rows to get good results.

Ted



Re: Row with many columns

2010-02-18 Thread Ted Zlatanov
On Thu, 18 Feb 2010 18:05:50 +0300 ruslan usifov  
wrote: 

ru> 2010/2/18 Ted Zlatanov 
>> Maybe you should partition your data differently?  What does each of
>> these tiny columns represent and how do you allocate the row key?

ru> Friends relations, for statistic

Sorry, that's not enough information.  Can you give an example of some
columns and what they mean, and also some row keys and what they mean?

Ted



Re: Row with many columns

2010-02-18 Thread Ted Zlatanov
On Thu, 18 Feb 2010 00:44:17 +0300 ruslan usifov  
wrote: 

ru> I have Table where 10 rows have 10 columns about 200 bytes in each
ru> column, So if read only this 10 records only nodes that have this rows does
ru> work, another nodes are idle. This bad, and cassandra doesn't provide any
ru> solution to solve this

Maybe you should partition your data differently?  What does each of
these tiny columns represent and how do you allocate the row key?

Ted



Re: "easy" interface to Cassandra

2010-02-02 Thread Ted Zlatanov
On Tue, 19 Jan 2010 08:09:13 -0600 Ted Zlatanov  wrote: 

TZ> My proposal is as follows:

TZ> - provide an IPluggableAPI interface; classes that implement it are
TZ>   essentially standalone Cassandra servers.  Maybe this can just
TZ>   parallel Thread and implement Runnable.

TZ> - enable the users to specify which IPluggableAPI they want and provide
TZ>   instantiation options (port, connection limit, etc.)

TZ> - write a simple HTTPPluggableAPI, which provides a web server and
TZ>   accepts POST requests.  The exact path and option spec can be worked
TZ>   out later.  The input and output formats can be specified with a query
TZ>   parameter; at least JSON and XML should be supported.

First very rough proposal is at
https://issues.apache.org/jira/browse/CASSANDRA-754

Ted



Re: [VOTE] Graduation

2010-01-26 Thread Ted Zlatanov
On Mon, 25 Jan 2010 15:11:28 -0600 Eric Evans  wrote: 

EE> I propose a vote for Cassandra's graduation to a top-level project.

Yes, definitely.

Ted



Re: How to unit test my code calling Cassandra with Thift

2010-01-25 Thread Ted Zlatanov
On Sun, 24 Jan 2010 13:56:07 +0200 Ran Tavory  wrote: 

RT> On Sun, Jan 24, 2010 at 1:16 PM, gabriele renzi  wrote:

>> On Sun, Jan 24, 2010 at 11:02 AM, Ran Tavory  wrote:
>> > Here's the code I've just written over the weekend and started using in
>> > test:
>> 
>> 
>> Thanks for sharing :)
>> A quick note on the code from a superficial look: instead of the
>> hardwired "tmp" string I think it would make more sense to use the
>> system's tmp dir (  System.getProperty("java.io.tmpdir")).
>> 
>> I'd say something like this deserves to be present in the cassandra
>> distribution, or at least put in some public repo (github,
>> code.google, whatever), what do other people think?

RT> agreed on the System.getProperty("java.io.tmpdir")
RT> I can put this under contrib if you think it's useful.

Maybe it would make sense to also add a constructor to XMLUtils to
accept a configuration directly from an InputStream instead of just a
String filename.  Then all these tmpdir games can be avoided.
DocumentBuilder, which is used behind the scenes, already does this so
it's a simple patch to add this constructor to XMLUtils.java:

public XMLUtils(InputStream xmlIS) throws ParserConfigurationException, 
SAXException, IOException
{
DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
DocumentBuilder db = dbf.newDocumentBuilder();
document_ = db.parse(xmlIS);

XPathFactory xpathFactory = XPathFactory.newInstance();
xpath_ = xpathFactory.newXPath();
}

Ted



Re: Too many open files

2010-01-22 Thread Ted Zlatanov
On Fri, 22 Jan 2010 11:27:09 +0100 Dr. Martin Grabmüller 
 wrote: 

MG> The obvious call to lsof did not give me any insight (with 2271 being
MG> my Cassandra instance's pid):

MG>   (env)cassan...@archive1:~$ lsof -p 2271|wc -l
MG>   101

MG> Maybe the file limit is reached while scanning all data files?

Looks like it, 100 is a nice round number (you're counting the header
too to get 101).  What does `ulimit -a' say?  You need to adjust it but
the way depends on the way your specific OS distro configures login.

Ted



Re: Cassandra with LongType columns

2010-01-21 Thread Ted Zlatanov
On Thu, 21 Jan 2010 15:42:51 -0600 Jonathan Ellis  wrote: 

JE> 2010/1/21 Sébastien Pierre :
>> Hi Jonathan,
>> Strangely, using the Java client I have no other option than using string,

JE> sure you do.  it takes a byte[], there are other ways of getting a
JE> byte[] than String.getBytes.

JE> the easiest way to get a byte[] with the contents of a long is to use
JE> a ByteBuffer.

This is what I use in a wider utility class.  I don't know if it's the
best way but it works well for me.

Ted

...
public static byte[] getLongAsBytes(long timestamp) throws IOException
{
ByteArrayOutputStream bos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(bos);
dos.writeLong(timestamp);
dos.flush();
return bos.toByteArray();
}

public static Long getLongFromBytes(byte[] data)
{
ByteArrayInputStream bis = new ByteArrayInputStream(data);
DataInputStream dis = new DataInputStream(bis);
try
{
return dis.readLong();
}
catch (IOException e)
{
return null;
}
}
...



Re: 'large' node configuration question

2010-01-21 Thread Ted Zlatanov
On Thu, 21 Jan 2010 14:47:41 -0600 Brandon Williams  wrote: 

BW> 2010/1/21 Ted Zlatanov 
>> Also, maybe these options:
>> 
>> -ea \
>> -Xdebug \
>> -XX:+HeapDumpOnOutOfMemoryError \
>> -Xrunjdwp:transport=dt_socket,server=y,address=,suspend=n \
>> 
>> should go in a "debugging" configuration, triggered by setting
>> $CASSANDRA_DEBUG?  With a 60+ GB heap, dumping it to a file could be
>> very painful.  It's pretty bad with a smaller heap too.

BW> You can always override the CASSANDRA_INCLUDE environment variable
BW> and point it at a file with your own options.

I suggested it as a convenience for the end users who don't want to
override CASSANDRA_INCLUDE and don't need the debugging options on by
default.  Not everyone is an expert on the JVM options.

Ted



Re: 'large' node configuration question

2010-01-21 Thread Ted Zlatanov
On Thu, 21 Jan 2010 11:04:58 -0600 Jonathan Ellis  wrote: 

JE> 2010/1/21 Ted Zlatanov :
>> Based on that, it seems like a good idea to enable the parallel or
>> concurrent garbage collectors with large heaps.  We're looking at this
>> at our site as well so I'm curious about people's experiences.

JE> Cassandra's default jvm options (bin/cassandra.in.sh) enable the concurrent 
GC.

On Thu, 21 Jan 2010 11:04:35 -0600 Brandon Williams  wrote: 

BW> Cassandra already uses the ParNew and CMS GCs by default (in 
cassandra.in.sh)

Are those the best GC choices for Cassandra on a machine like what the
OP mentioned?  There are many more tuning options:

http://java.sun.com/performance/reference/whitepapers/6_performance.html,
http://java.sun.com/javase/technologies/hotspot/gc/gc_tuning_6.html
http://blogs.sun.com/watt/resource/jvm-options-list.html

plus the specific settings Cassandra uses:

-XX:SurvivorRatio=8 \
-XX:TargetSurvivorRatio=90 \
-XX:+AggressiveOpts \
-XX:+UseParNewGC \
-XX:+UseConcMarkSweepGC \
-XX:+CMSParallelRemarkEnabled \
-XX:SurvivorRatio=128 \
-XX:MaxTenuringThreshold=0 \

may not be right for a heap 16-64 times larger than the 1 GB heap
specified in cassandra.in.sh.

Also, maybe these options:

-ea \
-Xdebug \
-XX:+HeapDumpOnOutOfMemoryError \
-Xrunjdwp:transport=dt_socket,server=y,address=,suspend=n \

should go in a "debugging" configuration, triggered by setting
$CASSANDRA_DEBUG?  With a 60+ GB heap, dumping it to a file could be
very painful.  It's pretty bad with a smaller heap too.

Finally, is there a reason the -server option is not used?

Thanks
Ted



Re: 'large' node configuration question

2010-01-21 Thread Ted Zlatanov
On Wed, 20 Jan 2010 21:14:27 -0600 Jonathan Ellis  wrote: 

JE> On Wed, Jan 20, 2010 at 9:10 PM, Phillip Michalak
JE>  wrote:
>> Does anyone have a recommendation for configuring cassandra on a cluster
>> with 'large' nodes?
>> i.e. multiple nodes, each >64G RAM, 8 cores, >4 disks
>> 
>> Would it be better to run multiple cassandra processes per node (configured
>> on separate IP addresses, separate data directories, etc) or just a single
>> mammoth cassandra process on each node? Is there an upper bound on what
>> you'd want to allocate to a cassandra process?

JE> assuming that you're planning to use most of the ram for read caching
JE> and not for the JVM process, I don't see any advantage to justify the
JE> extra complexity of running multiple processes per machine.

JE> (I only mention that caveat because I don't know how well the JVM
JE> scales to heaps that large.)

Sun mentions garbage collection issues in the HotSpot FAQ:
http://java.sun.com/docs/hotspot/HotSpotFAQ.html#64bit_description

Based on that, it seems like a good idea to enable the parallel or
concurrent garbage collectors with large heaps.  We're looking at this
at our site as well so I'm curious about people's experiences.

Ted



Re: "easy" interface to Cassandra

2010-01-19 Thread Ted Zlatanov
Ted Zlatanov wrote:
>>> I've got three (me, Tatu, Philip) in favor of such an interface.
>>> Jonathan, Eric, anyone else, WDYT?
>>> 
>>> My proposal is as follows:
>>> 
>>> - provide an IPluggableAPI interface; classes that implement it are
>>>  essentially standalone Cassandra servers.  Maybe this can just
>>>  parallel Thread and implement Runnable.
>>> 
>>> - enable the users to specify which IPluggableAPI they want and provide
>>>  instantiation options (port, connection limit, etc.)
>>> 
>>> - write a simple HTTPPluggableAPI, which provides a web server and
>>>  accepts POST requests.  The exact path and option spec can be worked
>>>  out later.  The input and output formats can be specified with a query
>>>  parameter; at least JSON and XML should be supported.

On Tue, Jan 19, 2010 at 6:13 AM, Jonathan Ellis  wrote:
> I'm a huge non-fan of the "let's specify everything in as minute
> detail as possible before writing anything" style that resulted in
> CASSANDRA-547 taking about ten times as long as necessary.  Code
> something up, if the approach has merit, it can be refined.

All right, thanks for the feedback.

Ted



Re: "easy" interface to Cassandra

2010-01-19 Thread Ted Zlatanov
On Mon, 18 Jan 2010 10:48:41 -0800 Tatu Saloranta  wrote: 

TS> 2010/1/15 Ted Zlatanov :
>> I could provide a way to plug in your own API to Cassandra so anyone
>> could write a full API using PB, HTTP, or whatever protocol they like.
>> Underneath it could invoke the Thrift interface methods locally and
>> relay their results out.  This would be really nice but again I doubt it
>> would be included in the core.

TS> I would find it useful to have such an alternative interface. I have
TS> nothing against Thrift, but there are cases where simpler interface
TS> would be useful. And in those cases it would be fine to have this as
TS> sort of adapter.

TS> For what it's worth, I think it would be nice to keep the two
TS> (transport over http, contents as json) separate, if possible. I like
TS> JSON, but there are use cases where other formats are better.
TS> For numeric data, for example, binary formats are more compact.

I've got three (me, Tatu, Philip) in favor of such an interface.
Jonathan, Eric, anyone else, WDYT?

My proposal is as follows:

- provide an IPluggableAPI interface; classes that implement it are
  essentially standalone Cassandra servers.  Maybe this can just
  parallel Thread and implement Runnable.

- enable the users to specify which IPluggableAPI they want and provide
  instantiation options (port, connection limit, etc.)

- write a simple HTTPPluggableAPI, which provides a web server and
  accepts POST requests.  The exact path and option spec can be worked
  out later.  The input and output formats can be specified with a query
  parameter; at least JSON and XML should be supported.

Ted



Re: "easy" interface to Cassandra

2010-01-15 Thread Ted Zlatanov
On Fri, 15 Jan 2010 09:23:08 -0800 Tatu Saloranta  wrote: 

TS> 2010/1/15 Ted Zlatanov :
>> Hell, let's make the query a RESTful HTTP GET, Cassandra a HTTP
>> server, and return the data as JSON if it's more palatable.  My point
>> is, string queries are a

TS> No contest there. :-)

If anyone, including you, is interested, I could put together a patch to
show how Cassandra would respond to REST HTTP queries with JSON-encoded
data.  Based on the feedback so far it seems like it wouldn't make it in
the core, though.

I could provide a way to plug in your own API to Cassandra so anyone
could write a full API using PB, HTTP, or whatever protocol they like.
Underneath it could invoke the Thrift interface methods locally and
relay their results out.  This would be really nice but again I doubt it
would be included in the core.

Ted



Re: "easy" interface to Cassandra

2010-01-15 Thread Ted Zlatanov
On Thu, 14 Jan 2010 14:34:58 -0800 Tatu Saloranta  wrote: 

TS> No specific proposal, or immediate need. But I do know that such
TS> short-hand notations / languages are popular for accessing structured
TS> data (xpath/xquery, oql, even sql).

Sure.  The idea is to make Cassandra more accessible.  I'll try to
explain all my annoyances with the current API and why I think something
simpler is useful, specifically string-based queries.

IMHO the current API is confusing and hard to use.  ColumnPath,
ColumnParent, and ColumnOrSuperColumn (with the semi-included Column and
SuperColumn) are not obvious until you've tried to use them and failed
at least a few times.  SliceRange works inconsistently between Perl and
Java, to give another example: Perl only has 64-bit numbers in the
minority of compilations, so you have to jump through a lot of hoops to
specify the current time as 8 bytes.  The Java call for a null key to
indicate an unbounded range uses "null" while Perl (and Python and Ruby
IIRC) have to use the empty string '' which is most definitely not the
null value.

The majority of Cassandra users seem to be Java programmers so maybe
these issues simply haven't mattered to them and the rest have put
together libraries like Lazyboy and EasyCassandra.pm.

TS> Just wanted to mention that most opposition seemed to be against
TS> specific way to use such notation. Not so much for idea of more
TS> convenient access, as long as it uses facilities host language
TS> offers.

The Thrift API is exposed to a wide variety of languages.  You simply
can't use OO the same way in Perl and Haskell as you do in Java, to give
two examples.  They are ridiculously different and my issue, really, is
with the ugliness that results and with the awkward code I've had to
write in order to accomodate the current API in Perl.  OO through Thrift
is simply not good OO.[1]

So coming back to the query language, you either simulate OO queries,
which Thrift already does as badly as can be expected, or you drop down
to multiple strings, which IMHO is a bad compromise, or you use a single
string like most universal APIs in existence.  Hell, let's make the
query a RESTful HTTP GET, Cassandra a HTTP server, and return the data
as JSON if it's more palatable.  My point is, string queries are a
legitimate way to use structured data and not a Perl-specific thing.

Ted

[1] Thrift has encapsulation, sort of, but not inheritance or
polymorphism which are essential for OO.  From the Thrift FAQ:

"Thrift Features

Is struct inheritance supported?

No, it isn't. Thrift support inheritance of service interfaces, but only 
composition/containment for structures.

Can I overload service methods?

Nope. Method names must be unique."

Just so it's clear, I'm not blaming Thrift, it's just a transport
mechanism.  From the days of CORBA and RMI to today's efforts (Google
Protocol Buffers and Thrift among others) it's always been like this.



Re: "easy" interface to Cassandra

2010-01-14 Thread Ted Zlatanov
On Wed, 13 Jan 2010 13:22:02 -0800 Tatu Saloranta  wrote: 

TS> I think there are 2 separate questions:

TS> (a) Would a path language make sense, and
TS> (b) How would that be exposed

TS> So I think more developers would be opposed to part (b) of exposing
TS> path queries using opaque things like Strings, in languages where
TS> other abstractions can be used.

TS> But it might be that the concept of path-based queries makes sense,
TS> just using more typed building blocks, to make access more natural for
TS> language in question.

Based on the feedback I got, I don't think anyone is interested.  If you
have a proposal, though, I'm all ears.  My "easy" solution was certainly
not the only possible one to simplify path queries.

Ted



Re: "easy" interface to Cassandra

2010-01-13 Thread Ted Zlatanov
On Wed, 13 Jan 2010 08:05:45 +1300 Michael Koziarski  
wrote: 

>> I see no value in pushing for ports of a Perl library to other
>> languages instead of allowing each to grow its own idiomatic one.

MK> That's definitely the way to go, the Easy.pm magic strings look a
MK> little like line noise to me ( a non-perler )

Thanks for the feedback.  I'll keep EasyCassandra Perl-only.

Ted



Re: "easy" interface to Cassandra

2010-01-12 Thread Ted Zlatanov
On Sun, 10 Jan 2010 11:16:20 + Mark Robson  wrote: 

MR> I can't see any reason to make an "easy" Cassandra interface, as the Thrift
MR> interface isn't really very difficult.

Compare this (this is what the "easy" interface would look like in Java,
wrapped in try/catch of course):

EasyCassandra client = new EasyCassandra("cassandraserver", "thekeyspace");
client.connect();
Map latest = client.get(new String[] { "row1" }, "Values/<-1>[]");

to the equivalent Thrift calls to connect to the server, specify you
want row "row1", supercolumn family "Values", last supercolumn, and get
all the columns as a Map.  I think this interface is significantly
easier.

MR> In any case the main problems with Cassandra will be design ones, i.e.
MR> figuring out how to use it effectively in your application. No "Easy"
MR> library is going to make that easier.

I agree that learning how to use Cassandra effectively is an issue, but
constructing the API calls is pretty laborious.  Especially in Perl it's
unnecessarily painful in my experience.  I think this presents a barrier
to adoption.

For my work, the "easy" interface has shortened my code, allowed me to
write quick one-liners for targeted data extraction, and enabled me to
experiment with Cassandra more freely.

If no one else sees value in it, I'll keep the "easy" interface as a
Perl module and release on CPAN.  Can I get some more opinions?

Thanks
Ted



"easy" interface to Cassandra (was: EasyCassandra.pm Perl interface alpha 0.01)

2010-01-09 Thread Ted Zlatanov
I was wondering if it would make sense to add the pseudo-language
EasyCassandra.pm uses right into Cassandra and expose it over Thrift.

Here's a summary of the requests supported by this language:

# read and remove requests:
# "X/[Y][A,B]": supercolumn family X, super column Y (not a timestamp), 
columns A and B
# "X/[Y][]": supercolumn family X, super column Y (not a timestamp), 
all columns

# "X/{1234567890}[A,B]": supercolumn family X, timestamp numeric super 
column (packed to 8 bytes), columns A and B
# "X/{1234567890}[]": supercolumn family X, timestamp numeric super 
column (packed to 8 bytes), all columns

# "X/{}[A,B]": supercolumn family X, all the super columns, columns A 
and B (but all will be returned or removed)
# "X/{}[]": supercolumn family X, all the super columns, all columns

# "X[]": column family X, all columns
# "X[A,B]": column family X, columns A and B

# super column ranges work as expected, with commas:
# "X/[Y,Z][]": supercolumn family X, super columns Y and Z (not a 
timestamp), all columns
# "X/{1234567890,1234567891}[A,B]": supercolumn family X, two timestamp 
numeric super column (packed to 8 bytes), columns A and B

# read only requests (positional requests are only for reading)
# "X/<0>[A,B]": supercolumn family X, first super column, columns A and 
B (but all will be returned)
# "X/<0>[]": supercolumn family X, first super column, all columns

# "X/<4>[A,B]": supercolumn family X, fourth super column, columns A 
and B (but all will be returned)
# "X/<4>[]": supercolumn family X, fourth super column, all columns

# "X/<-3>[A,B]": supercolumn family X, last three super columns, 
columns A and B (but all will be returned)
# "X/<-3>[]": supercolumn family X, last three super columns, all 
columns

# "X/<-1>[A,B]": supercolumn family X, last super column, columns A and 
B (but all will be returned)

# write requests:
# "X/Y[Z]": supercolumn family X, super column Y (not a timestamp), 
column Z
# "X/{1234567890}[Z]": supercolumn family X, timestamp numeric super 
column (packed to 8 bytes), column Z
# "X/{}[Z]": supercolumn family X, current timestamp numeric super 
column (packed to 8 bytes), column Z
# "X[Y]": column family X, column Y

All the data is passed around with maps.  While this won't work for
every use case, I think for a large number it will be sufficient.  At
least for my needs it has worked very well.  Especially for scripting
languages like Perl, Ruby, or Python, this is a convenient way to write
one-liners: you don't need the whole Thrift class hierarchy, just the
get/set/remove calls, and you don't need to worry about ranges, which
makes the client code simpler (results can be returned iterator-style if
performance is a concern).  In fact this can be a second Thrift
interface to Cassandra, also implemented by CassandraServer but
completely independent of the primary so each of the two interfaces
won't be affected by any changes to the other.

If this is interesting I can put a prototype together with ANTLR.

Ted



latest auth patch available in https://issues.apache.org/jira/browse/CASSANDRA-547

2009-12-29 Thread Ted Zlatanov
This patch adds auth support as previously discussed, while also
patching Cassandra to support Thrift's new constructors.  Only the
AllowAll backend is currently provided.

Note a newer libthrift, also uploaded there, is required.  I have not
tested backwards compatibility of old Thrift clients with the new
libthrift.

Thanks
Ted



Re: Partition data - advantage and disadvantage

2009-12-28 Thread Ted Zlatanov
On Mon, 28 Dec 2009 11:44:27 -0700 Joe Stump  wrote: 

JS> On Dec 28, 2009, at 11:40 AM, Ted Zlatanov wrote:

>> I can see that's a problem.  In my case, row keys represent switches in
>> production so I don't expect more than a few hundred.  An application
>> can't find out how many switches are known without enumerating the
>> keys; how would you suggest I attack the problem?

JS> If you only have a few hundred why not do a single key range query
JS> with a limit of 5000?

That makes sense, I was fixated on getting all the keys in one call.
Thanks.

Ted



Re: Partition data - advantage and disadvantage

2009-12-28 Thread Ted Zlatanov
On Mon, 28 Dec 2009 11:02:30 -0700 Joe Stump  wrote: 

JS> On Dec 28, 2009, at 11:00 AM, Ted Zlatanov wrote:

>> Is this worth a JIRA feature request?  Or is it something Cassandra will
>> never support fully?  From the user's perspective it's very useful.

JS> I don't know why it'd be very useful to be honest. Lots of us have
JS> CF's with billions of keys. Ours, for instance, tends to shit its
JS> pants when returning ~300k keys.

I can see that's a problem.  In my case, row keys represent switches in
production so I don't expect more than a few hundred.  An application
can't find out how many switches are known without enumerating the
keys; how would you suggest I attack the problem?

JS> If you're asking for a CF count value, I can see that being useful
JS> (e.g. There are 4m keys in CF 'foo').

That's more useful for your setup, absolutely.  In my case the number is
too small to matter.

Ted



Re: Partition data - advantage and disadvantage

2009-12-28 Thread Ted Zlatanov
On Mon, 28 Dec 2009 09:53:56 -0700 Joe Stump  wrote: 

JS> On Dec 28, 2009, at 9:51 AM, Ted Zlatanov wrote:

>> If each node does a key enumeration, can the results be aggregated
>> somehow?  It seems useful to get a list of all the keys across the
>> cluster even if it's not 100% accurate.  I didn't see discussions of
>> such a feature in JIRA or in the archives of this list.

JS> The results are aggregated AFAIK. If you do a key range and keys are on 3 
nodes it'll return a cohesive list. It's much more tricky to get a list of ALL 
of the keys across the cluster. We've found that you need to do many key ranges 
to enumerate over all of the keys in the cluster. 

JS> It's possible, just not wrapped in a simple function.

Is this worth a JIRA feature request?  Or is it something Cassandra will
never support fully?  From the user's perspective it's very useful.

Thanks
Ted



Re: Partition data - advantage and disadvantage

2009-12-28 Thread Ted Zlatanov
On Mon, 28 Dec 2009 08:07:18 -0700 Joe Stump  wrote: 

JS> The advantage of the random partitioner is that it randomly
JS> distributes your keys across the cluster. This (theoretically)
JS> avoids key clustering on nodes. The big disadvantage is that you
JS> can't do key range operations.

If each node does a key enumeration, can the results be aggregated
somehow?  It seems useful to get a list of all the keys across the
cluster even if it's not 100% accurate.  I didn't see discussions of
such a feature in JIRA or in the archives of this list.

Ted



EasyCassandra.pm Perl interface alpha 0.01

2009-12-28 Thread Ted Zlatanov
Attached is the first alpha (0.01) version of my EasyCassandra.pm Perl
interface to Cassandra.  I am also attaching a demo script that will
show the intended usage, but basically the idea is that the user can
express gets, puts, and removals in shorthand like "Subscribed/<-1>[]"
to mean the latest Subscribed super column, all columns.  Please look at
the easydemo.pl script to see the exact syntax; the EasyCassandra.pm
module has a full list of all the acceptable specifications.

There are some TODO items and I can probably optimize the code so it
doesn't use multiget_slice() for everything.  Also the code to retrieve
all row keys and all the super columns (when they are more than the
usual 100 limit) is hacky.  Nevertheless, this is IMHO about 10x easier
to use than the auto-generated Perl modules, and 5x easier than
Net::Cassandra on CPAN.

You'll need Moose, Class::Accessor, Parse::RecDescent and Hash::Merge
from CPAN, plus all the Thrift auto-generated Perl code.

For those who are interested, the grammar generates all the parameters
for removals and insertions on the fly as it parses the specification.
It wasn't as easy to generate read parameters on the fly so I
post-generate those from the parse tree.

Consider this an alpha, released for public testing before I upload it
to CPAN in a week or two.  I would appreciate any suggestions or
comments, especially on the grammar and API usage.

Thanks
Ted

#!/usr/local/jump/perl/bin/perl

package EasyCassandra;

use Moose;
use warnings;
use strict;

use lib qw(./cassandra ./cassandra/Cassandra /home/tzz/workspace/HeartbeatMonitor2/cassandra /home/tzz/workspace/HeartbeatMonitor2/cassandra/Cassandra);
use Data::Dumper;

use Class::Accessor;

use Cassandra;
use Thrift::Socket;
use Thrift::BinaryProtocol;
use Thrift::FramedTransport;
use Thrift::BufferedTransport;


use Parse::RecDescent;
use Hash::Merge;

# version 0.01 alpha

# TODO: version properly
# TODO: set up import paths correctly
# TODO: use Time::HiRes for better timestamps
# TODO: use a better test module for unit testing (maybe against a live Cassandra server)

# use Time::HiRes qw( gettimeofday tv_interval );

Hash::Merge::set_behavior( 'RETAINMENT_PRECEDENT' );

our $DEBUG = 0;

# plain options, required for construction
has server   => ( is => 'ro', isa => 'Str', required => 1 );
has keyspace => ( is => 'ro', isa => 'Str', required => 1 );

# plain options with defaults
has port => ( is => 'ro', isa => 'Int', default => 9160 );
has recv_timeout => ( is => 'ro', isa => 'Int', default => 5000 );
has send_timeout => ( is => 'ro', isa => 'Int', default => 1000 );
has recv_buffer  => ( is => 'ro', isa => 'Int', default => 1024 );
has send_buffer  => ( is => 'ro', isa => 'Int', default => 1024 );
has max_results  => ( is => 'ro', isa => 'Int', default => 100 );

# read and write consistency can be changed on the fly
has read_consistency  => ( is => 'rw', isa => 'Int', default => Cassandra::ConsistencyLevel::ONE );
has write_consistency => ( is => 'rw', isa => 'Int', default => Cassandra::ConsistencyLevel::ONE );

# internals
has socket=> (is => 'rw', isa => 'Thrift::Socket');
has protocol  => (is => 'rw', isa => 'Thrift::BinaryProtocol');
has client=> (is => 'rw', isa => 'Cassandra::CassandraClient');
has transport => (is => 'rw', isa => 'Thrift::BufferedTransport');
has opened=> (is => 'rw', isa => 'Bool');

use constant THRIFT_MAX => 100;

use constant GRAMMAR_SPECIAL => 'special';
use constant GRAMMAR_EXACT   => 'exact';
use constant GRAMMAR_ALL => 'ALL';

our $last_predicate = Cassandra::SlicePredicate->new({
		  slice_range => Cassandra::SliceRange->new({start=> '' , finish=> '', reversed => 1, count => 1}),
		 });

our $first_predicate = Cassandra::SlicePredicate->new({
		   slice_range => Cassandra::SliceRange->new({start=> '' , finish=> '', reversed => 1, count => 1}),
		  });

our $all_predicate = Cassandra::SlicePredicate->new({
		 slice_range => Cassandra::SliceRange->new({start=> '' , finish=> ''}),
		});

$Parse::RecDescent::skip = '';
our $grammar = new Parse::RecDescent(q{

# read and remove requests:
	# "X/[Y][A,B]": supercolumn family X, super column Y (not a timestamp), columns A and B
	# "X/[Y][]": supercolumn family X, super column Y (not a timestamp), all columns

	# "X/{1234567890}[A,B]": supercolumn family X, timestamp numeric super column (packed to 8 bytes), columns A and B
	# "X/{1234567890}[]": supercolumn family X, timestamp numeric super column (packed to 8 bytes), all columns

	# "X/{}[A,B]": supercolumn family X, all the super columns, columns A and B (but all will be returned or removed)
	# "X/{}[]": supercolumn family X, all the super columns, all columns

	# "X[]": column family X, all columns
	# "X[A,B]": column family X, columns A and B

# super column ranges work as expected, with commas:
	# "X/[Y,Z][]": supercolumn family X, super columns Y and Z (not a timestamp), all columns
	# "X/{12

Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 2 Dec 2009 15:32:35 -0600 Jonathan Ellis  wrote: 

JE> 2009/12/2 Ted Zlatanov :
>> I'd still rather pass something back.  As I said, it allows backends to
>> maintain state when it makes sense to do so and can alleviate the
>> problem of redundant auth queries in the future.

JE> That makes no sense whatsoever.  Backends can maintain state or not
JE> either way; adding a token you have to pass back makes just adds an
JE> extra layer of mapping token -> real state in the simple case of
JE> token-is-only-valid-per-connection and an unreasonable amount of
JE> complexity if you try to make it valid across more than one.

JE> I'm -1 in the apache veto sense on the token idea.

On Wed, 02 Dec 2009 15:38:14 -0600 Eric Evans  wrote: 

EE> If not for SSO via shared-state between nodes, then for what? Can you
EE> give a tangible example of "when it makes sense to do so"? Are you sure
EE> this isn't YAGNI? 

Across nodes, backends can't maintain state easily, and even across
threads it's not trivial.  My version allows a distrubuted application
to authenticate once and then reuse the same AuthenticationRequest, as
long as the authentication backends have a permanent token store inside
or outside Cassandra.  If we're going to make the keyspace choice sticky
per connection, we should allow for a way to open multiple connections
to multiple keyspaces without necessarily hitting auth services every
time.

You claim the complexity is unreasonable.  I disagree.  If any
complexity emerges, it will be in the backend code which a particular
organization may need and will write.  For Cassandra itself this is not
an issue and the default case (AllowAll) will not create any such
complexity.

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 2 Dec 2009 15:23:23 -0600 Jonathan Ellis  wrote: 

JE> It's really premature to be holding a "vote" based on
JE> first-impression opinions.

Somehow we have to make a decision on whether the API will be stateful
or stateless.  This affects more than just the auth code so I thought a
vote was sensible.  How else can we resolve the debate so I can actually
write some code?  I really don't have a preference as I mentioned, I
just want to get the auth code done.

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 02 Dec 2009 14:35:09 -0600 Eric Evans  wrote: 

EE> On Wed, 2009-12-02 at 14:27 -0600, Ted Zlatanov wrote:
>> On Wed, 02 Dec 2009 14:14:53 -0600 Eric Evans  wrote: 
>> 
EE> Did you maybe mean...?  AuthenticationRequest required for the
EE> method (has to be), but the map is optional?
>> 
>> Either way will work.  With your suggestion and allowing for auth
>> backends to modify the authentication request:

EE> Let's not do that; as Robin pointed out...

EE> "...there's no way to ensure immediate consistency of tokens across all
EE> nodes, it's probably not worth the effort of making Cassandra "token
EE> aware", since an application would have to provide fall-back logic to
EE> authenticate with credentials, if the token fails.  So, tokens would
EE> probably just add complexity without much benefit."

I'd still rather pass something back.  As I said, it allows backends to
maintain state when it makes sense to do so and can alleviate the
problem of redundant auth queries in the future.

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 2 Dec 2009 20:54:13 + Mark Robson  wrote: 

MR> How about we make authentication optional, and have the protocol being
MR> stateful only if you want to authenticate?

MR> That way we don't break backwards compatibility or introduce extra
MR> complexity for people who don't need it.

That was my original proposal.  Jonathan and Eric disagreed and made
good arguments against it.  Multiple operation modes are more confusing
IMO.

In the latest proposed version, authentication is off by default: all
calls go to the AllowAll backend with a negligible overhead.  You still
have to make a login() call but it's effectively a setKeyspace().

I don't think backwards compatibility should limit innovation in a 0.x
release.  The extra complexity is negligible: you just pass an empty
AuthenticationRequest if you don't need it.  From that point on, you
actually have one less argument (no keyspace) on many Thrift calls.

On Wed, 2 Dec 2009 15:59:40 -0500 Jake Luciani  wrote: 

JL> +1 this is nosql afterall.

I don't think NoSQL == stateless.  The two are completely orthogonal.

The vote is 3-3 (Jonathan, Robin, Evan - Mark, Jake, Bill).

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 02 Dec 2009 14:14:53 -0600 Eric Evans  wrote: 

EE> Did you maybe mean...?  AuthenticationRequest required for the
EE> method (has to be), but the map is optional?

Either way will work.  With your suggestion and allowing for auth
backends to modify the authentication request:

struct AuthenticationRequest {
1: optional map credentials,
}

service Cassandra {
...
  # authentication and authorization.  Returns the authentication request 
(never null) with any auth tokens that may have been created by the auth 
backends 
  AuthenticationRequest login(1: required string keyspace, 
  2: required AuthenticationRequest auth_request) 
throws (1:AuthenticationException aux, 2: 
AuthorizationException azx),
...
}

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 02 Dec 2009 19:13:58 + Bill de hOra  wrote: 

Bdh> Strong agreement. The login method should return an opaque token which
Bdh> is passed on each call. If that messes up the API, that says more
Bdh> about Thrift than the auth model.

Slight disagreement on the semantics: this is not a stateless protocol
anymore.  Both the client and the server now need to keep track of these
tokens and set expiration policies on them, throw token validity
exceptions, etc.  It's useful for sure, but not stateless.  I'll put you
down for stateless but you may want to comment further.

So the question then flows into Robin's note:

On Wed, 2 Dec 2009 11:59:57 -0800 "Coe, Robin"  wrote: 

CR> Once a connection is opened with credentials, then as long as I hold
CR> that connection open, I shouldn't need to pass auth checks with
CR> every transaction.

Right, so you're on the stateful side.  With Bill's vote we're 3-2 in
favor of stateful.

CR> On the other hand, if there was a way to use a token in lieu of
CR> credentials, to provide SSO capabilities to any node, then I could
CR> see their use.  Checking whether a token exists in a local
CR> collection would be much faster than having to pass through an
CR> authentication/authorization framework.  However, as there's no way
CR> to ensure immediate consistency of tokens across all nodes, it's
CR> probably not worth the effort of making Cassandra "token aware",
CR> since an application would have to provide fall-back logic to
CR> authenticate with credentials, if the token fails.  So, tokens would
CR> probably just add complexity without much benefit.

The credentials are a map so it's trivial to add any credential (e.g. a
GPG signed keyspace authorization request, Kerberos tokens, or
Cassandra's own tokens).  I think each AuthenticationBackend
implementation should implement its own scheme (being careful not to
overlap map keys), so certainly SSO is a possibility.  Of course you
should be able to have an AuthenticationOrBackend that combines two or
more other backends, so it would for example try tokens first and then
fall back to the slower credentials.  The application wouldn't have to
provide any fall-back logic.  To accomplish this, though, the
credentials will have to be returned by the login() method at the Thrift
level.  I'll make the change.

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Wed, 2 Dec 2009 15:13:11 + Mark Robson  wrote: 

MR> I'd be against moving to a stateful protocol.

Noted, I'd like to see some more votes.  I'm agnostic: I think it will
work fine either way.  Eric and Jonathan are definitely on the stateful
side so it's 2-1 right now.  I think Robin is also on the stateful side
from his earlier notes so it may be 3-1.

On Wed, 02 Dec 2009 11:59:56 -0600 Eric Evans  wrote: 

EE> I'm thinking...

EE> void login(1:required string keyspace, 2:optional map
EE> authentication) throws AuthenticationException, AuthorizationException

EE> You're always going to want the keyspace supplied, but depending on how
EE> the cluster is configured, you may not need anything else (so make the
EE> map the second argument, and make it optional).

I was going to use an empty map to signify no credentials.  Optional
works just as well.  Thus the Thrift definition is:

# invalid authentication request (user does not exist or credentials invalid)
exception AuthenticationException {
1: required string why
}

# invalid authorization request (user does not have access to keyspace)
exception AuthorizationException {
1: required string why
}

struct AuthenticationRequest {
1: required map credentials,
}

service Cassandra {
...
  void login(1: required string keyspace, 
 2: optional AuthenticationRequest auth_request) 
   throws (1:AuthenticationException aux, 2: AuthorizationException azx),
...
}

Ted



Re: Cassandra access control

2009-12-02 Thread Ted Zlatanov
On Tue, 01 Dec 2009 16:58:50 -0600 Eric Evans  wrote: 

EE> On Tue, 2009-12-01 at 15:38 -0600, Ted Zlatanov wrote:
>> I disagree, why would you want to forbid switching the keyspace?  That's
>> turning off a currently working feature.  Also, connections are not
>> free, especially across WAN links.

EE> Because it makes a mess out of an API that already confuses people. And
EE> again, I posit that this is a corner case, something that very few
EE> people will actually want to do. We should be optimizing for the common
EE> case.

On Tue, 1 Dec 2009 17:13:24 -0600 Jonathan Ellis  wrote: 

JE> It's a currently working feature that (a) only exists in the first
JE> place because it was less bad than postponing 0.4 until we could add
JE> something like this auth feature, and (b) is of entirely theoretical
JE> benefit in the sense that nobody who wants to keep it actually has an
JE> app using it, to the best of my knowledge.

JE> Let's correct our error, and only add functionality that we know is
JE> really needed -- and even then, only if the benefit outweighs the
JE> complexity price -- rather than waving our hands and saying "maybe
JE> someone will need it, someday."  That way leads to clutter and bloat.

OK.  So what should the API be?  Just one method, as Robin suggested?

void login( Map credentials, String keyspace ) 
 throws AuthenticationException, AuthorizationException

In this model the backend would still have login() and
setKeyspace()/getKeyspace() separately to distinguish between
authentication and authorization but the frontend API would merge them.

This means the keyspace parameter will be removed from all the current
Thrift API calls and application will need to remember what keyspace
they requested.  It will definitely not make 0.5.

Let me know if this is OK and I'll prepare a new patch.

Thanks
Ted



Re: Cassandra access control

2009-12-01 Thread Ted Zlatanov
On Tue, 01 Dec 2009 14:23:47 -0600 Eric Evans  wrote: 

EE> I'm personally not a big fan of the setKeyspace()/getKeyspace() idea.
EE> Getting rid of the keyspace argument makes sense because the keyspace is
EE> the highest (or lowest), level of the data-model so its implicit that an
EE> application need only talk to one. If removing that argument makes sense
EE> (and I think it does), then we should just do that. Accessing more than
EE> one keyspace is a corner case IMO, but its one that can be accommodated
EE> by opening another connection.

I disagree, why would you want to forbid switching the keyspace?  That's
turning off a currently working feature.  Also, connections are not
free, especially across WAN links.

EE> Also, I don't know how others feel, but I'm not comfortable committing
EE> anything that would be a candidate for 0.5 that isn't more fully baked
EE> at this point. So I'd rather see something that is at least minimally
EE> useful targeted for 0.5+1 

I'm not dying to get into 0.5.  My goal is to get a good API in place.
First thing is to agree on the API, then I can implement it.  My first
attempt simply keeps everything working like it did before by using the
AllowAll authenticator.  The next step is to put in configuration
support to use a different authenticator and provide a few basic ones as
we discussed (PAM and LDAP probably).  It's unlikely we'll make 0.5
considering its current state is close to release IIUC.

If we miss 0.5 I'll take out the backwards compatibility API that still
allows the keyspace argument for a few of the calls.  It's not a big
deal to do so.

Ted



Re: Cassandra access control

2009-11-30 Thread Ted Zlatanov
I posted the first attempt (with a default AllowAll backend) as a patch:

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

Important changes in the Thrift interface:

# invalid authentication request (user does not exist or credentials invalid)
exception AuthenticationException {
1: required string why
}

# invalid authorization request (user does not have access to keyspace)
exception AuthorizationException {
1: required string why
}

...

struct AuthenticationRequest {
1: required map credentials,
}

service Cassandra {
  # auth methods
  # authentication
  void login(1:required AuthenticationRequest auth_request) throws 
(1:AuthenticationException aux),
  
  # authorization
  void setKeyspace(1:required string keyspace) throws (1:AuthorizationException 
azx),
  string getKeyspace(),
...

When I generated the Java files, I got compilation errors on some
unrelated stuff but the auth code will at least compile, and should be
100% backwards compatible as discussed.  I only looked for the string
"keyspace" so I may have missed some places.  Please let me know what
you think.

Thanks
Ted



Re: Cassandra access control

2009-11-27 Thread Ted Zlatanov
On Wed, 25 Nov 2009 09:05:51 -0800 "Coe, Robin"  wrote: 

CR> If all you want to perform is a simple bind to an LDAP service, then
CR> why use either?  JPam uses JAAS under the covers and jldap is a full
CR> API for managing a depot.  Neither solution looks particularly
CR> optimized.

I thought (from the docs) that JPam was like SysAuth, an actual library
to talk to PAM instead of a JAAS overlay.  JAAS as we discussed only
works for the user that's running Cassandra, so it's not a good option.

jldap seems (again, from the docs) to support secure LDAP better than
JNDI (see below).

CR> If ldap integration is a must-have, then why not just use JNDI?
CR> Create a singleton factory that sets up the environment, including a
CR> connection pool, to create an initial context.  Then, use that to
CR> create a per-Thrift connection  binding context, with credentials
CR> passed in from the client?

I considered that too.  Based on
http://forums.sun.com/thread.jspa?threadID=581425&tstart=45 secure LDAP
is a pain to set up with JNDI and I'm pretty sure most LDAP users will
want it.  What's your experience with that?

CR> However, I still think the simplest, fastest solution is to use a
CR> Cassandra-managed user realm, similar to RDBMS systems.  That keeps
CR> the connection opening phase within the Cassandra engine and isn't
CR> susceptible to the service being unavailable.  As well, if Cassandra
CR> manages the user realm on a per-keyspace basis, then authentication
CR> and authorization can be performed simultaneously and the keyspace
CR> argument can be dropped from the Thrift API calls.

I'll provide a mechanism to give a user name and a password directly in
the configuration per keyspace as you suggest.  There will also be an
AllowAll backend to allow any access.  See
https://issues.apache.org/jira/browse/CASSANDRA-547

The password will probably be MD5 hashed in transit (with some salt).
MD5 is significantly faster than the more secure alternatives and I
think, when salted, it's more than reasonably secure for one-way
hashing.  I'm not a cryptography expert, though, so I'm open to
suggestions.

I think Jonathan agreed with me that a later version of the Thrift API
will drop the keyspace argument in favor of a setKeyspace() call.  For
now (phase 1) the two will coexist.

CR> Configuring Cassandra to handle LDAP binding will require
CR> configuring the connection url, protocol, search scope, base DN,
CR> keystore file, etc.  And of course, if Cassandra has LDAP
CR> integration, it should probably offer other authentication service
CR> support, like RADIUS and TACACS+, etc.  It's a can of worms, to be
CR> sure.

If it was easy it would have been done already :)  I appreciate your
help and we'll get this done soon.  I've spent a lot of time on
libraries, unfortunately.

CR> While on the topic of authentication, I still like the idea of
CR> opening a connection with credentials, as opposed to requiring a
CR> separate transaction to login.  That's an unnecessary round trip.  I
CR> don't see why an overloaded method to connect is a bad thing,
CR> especially when the anonymous connection will eventually be
CR> deprecated.  At least, I assume it will be deprecated by the time
CR> Cassandra has a fully fleshed out authentication realm?

We can do that as one API call, as long as internally Cassandra splits
it in two.  Also the user has to know that the method could return more
than one kind of exception (connection and authentication/authorization
problems can happen in one call).

The anonymous connection will simply map to the AllowAll backend I
mentioned.  Any username and password will work.  There will be no way
to establish a connection or set a keyspace without using a backend, so
in that sense anonymous connects will be deprecated, but they won't go
away.

It should be easy for the user to provide their own backend, similar to
the partitioner approach in Cassandra.  There's no reason to block
organizations from implementing their own auth schemes.  Cassandra will
simply map a keyspace to a backend and let the backend do all the auth
work.

Ted



Re: Cassandra access control

2009-11-24 Thread Ted Zlatanov
Looks like I could use:

PAM auth: http://jpam.sourceforge.net/

LDAP/AD auth: http://www.openldap.org/jldap/

The first is definitely OK (Apache license), but I'm not sure about the
second one (OpenLDAP public license).  Looks BSDish to me.  It claims to
support Windows auth and is officially provided by the OpenLDAP project.
Has anyone used it?

Thanks
Ted



Re: Wish list [from "users survey" thread]

2009-11-24 Thread Ted Zlatanov
On Tue, 24 Nov 2009 11:44:03 -0500 Ian Holsman  wrote: 

IH> I'd like to see how many times a specific user hits the site,
IH> without having to add them up every time.

Please excuse any inaccuracies here, I don't know Cassandra as well as
Jonathan and many others here.

Looking at the source, get_count over a super column seems to be
inefficient for this purpose (in
CassandraServer.java::multigetCountInternal()).  Would optimizing that
solve your need?  I'd guess that even ConsistencyLevel.QUORUM can be
pretty efficient if this is optimized (modulo the network latency), but
ConsistencyLevel.ONE can definitely be faster if the intermediate
columns are not created.

I only found https://issues.apache.org/jira/browse/CASSANDRA-329 to be
slightly related to this, so I don't think it's been considered before.

Ted



Re: Wish list [from "users survey" thread]

2009-11-24 Thread Ted Zlatanov
On Mon, 23 Nov 2009 13:45:09 -0600 Jonathan Ellis  wrote: 

JE> 1. Increment/decrement: "atomic" is a dirty word in a system
JE> emphasizing availability, but incr/decr can be provided in an
JE> "eventually consistent" manner with vector clocks.  There are other
JE> possible approaches but this is probably the best fit for us.  We'd
JE> want to allow ColumnFamilies with either traditional (for Cassandra)
JE> long timestamps, or vector clocks, but not mixed.  The bad news is,
JE> this is a very substantial change and will probably not be in 0.9
JE> unless someone steps up to do the work.  (This would also cover
JE> "flexible conflict resolution," which came up as well.)

Just for my benefit, can someone explain the reasons why atomic inc/dec
are needed inside Cassandra if 64-bit time stamps and UUIDs are
available?  I have not needed them in my usage but am curious about
other schemas that do.

Thanks
Ted



Re: cassandra over hbase

2009-11-24 Thread Ted Zlatanov
On Mon, 23 Nov 2009 11:58:08 -0800 Jun Rao  wrote: 

JR> After chatting with some Facebook guys, we realized that one potential
JR> benefit from using HDFS is that the recovery from losing partial data in a
JR> node is more efficient. Suppose that one lost a single disk at a node. HDFS
JR> can quickly rebuild the blocks on the failed disk in parallel. This is a
JR> bit hard to do in cassandra, since we can't easily find the data on the
JR> failed disk from another node. 

This is an architectural issue, right?  IIUC Cassandra simply doesn't
care about disks.  I think that's a plus, actually, because it
simplifies the code and filesystems in my experience are better left up
to the OS.  For instance, we're evaluating Lustre and for many specific
reasons it's significantly better for our needs than HDFS, so HDFS would
be a tough sell.

JR> So, when this happens, the whole node probably has to be taken out
JR> and bootstrapped. The same problem exists when a single sstable file
JR> is corrupted.

I think recovering a single sstable is a useful thing, and it seems like
a better problem to solve.

Ted



Re: Cassandra users survey

2009-11-24 Thread Ted Zlatanov
On Mon, 23 Nov 2009 23:30:51 -0500 Matt Revelle  wrote: 

MR> Are you both using timestamps as row keys?  Would be great to hear
MR> more details.

I'm using super column keys in a super column.

So let's say your resource is "routerA."

Your data will be:

Row "routerA"
 SuperColumn "Status"
  SuperColumn key T0 (this morning)
   Columns { status: connected, location: USA, ... }
  SuperColumn key T1 (T0 + 10 seconds for example)
   Columns { status: disconnected, location: Europe, ... }
  SuperColumn key T2 (T1 + 10 seconds for example)
   Columns { status: connected, ... } // no location specified

Then you can say "give me the latest super column key" (limit = 1, 
order = reversed, start == end == 0) and you'll get T1.

Ted



Re: Cassandra access control

2009-11-23 Thread Ted Zlatanov
On Mon, 23 Nov 2009 12:22:37 -0600 Jonathan Ellis  wrote: 

JE> sysauth says it is GPL v2 (also not compatible)

Hmm.  I guess I have to reimplement SysAuth.  At least the code is not
terribly complicated, but it's a shame to reinvent the cart and the wheel.

Ted



Re: Cassandra access control

2009-11-23 Thread Ted Zlatanov
On Fri, 20 Nov 2009 15:22:07 -0600 Jonathan Ellis  wrote: 

JE> Kasai is LGPL, and thus not compatible w/ Cassandra.  (See
JE> http://www.apache.org/legal/3party.html)

How annoying, it was exactly what I needed.  I hate reinventing the
wheel.  I can at least use SysAuth
(www.scribblin.gs/software/sysauth.html), I think.

Ted



Re: Cassandra users survey

2009-11-23 Thread Ted Zlatanov
On Fri, 20 Nov 2009 17:38:39 -0800 Dan Di Spaltro  
wrote: 

DDS> At Cloudkick we are using Cassandra to store monitoring statistics and
DDS> running analytics over the data.  I would love to share some ideas
DDS> about how we set up our data-model, if anyone is interested.  This
DDS> isn't the right thread to do it in, but I think it would be useful to
DDS> show how we store billions of points of data in Cassandra (and maybe
DDS> get some feedback).

I'd like to see that.  My Cassandra use is also for monitoring and so
far it has been great.  I store status updates in a SuperColumn indexed
by date and each row represents a unique resource.  It's really simple
compared to your setup, I'm sure.

Ted



Re: Cassandra access control

2009-11-20 Thread Ted Zlatanov
On Thu, 12 Nov 2009 12:09:19 -0600 Ted Zlatanov  wrote: 

TZ> I created an issue:

TZ> https://issues.apache.org/jira/browse/CASSANDRA-547

TZ> and will post updates there as needed.  This is stage 1, meaning this is
TZ> the 0.5 work that will keep the old API.  Stage 2 will remove the
TZ> Keyspace parameters from the API so I'll put that work in a different
TZ> issue, linked to CASSANDRA-547.

I found a good open-source library for auth*:
http://kasai.manentiasoftware.com/

Has anyone used it?  Looking at the source and API docs, it's decent and
supports several backends out of the box, including MySQL, PostgreSQL,
and SQL Server (which covers Sybase too).  I'd rather use it than put my
own together :)

It uses http://www.scribblin.gs/software/sysauth.html to provide a PAM
interface (which can then hook into LDAP/AD/etc. securely); an example
of SysAuth use with JBoss is at
http://www.mrchucho.net/2004/12/27/jboss-linux-authentication/

Unless there are any objections I'll use Kasai.

Ted



Re: out of memory error on malformed Thrift protocol

2009-11-13 Thread Ted Zlatanov
On Fri, 13 Nov 2009 13:57:38 -0600 Michael Greene  
wrote: 

MG> 2009/11/13 Ted Zlatanov :
>> The sequence to trigger the bug:
>> 
>> 1) telnet to port 9160
>> 
>> 2) type:
>> 
>> s
>> s
>> 
>> (two letter+RET combinations)

MG> This is a know issue and is out of Cassandra's specific hands.  The
MG> Thrift issue is: http://issues.apache.org/jira/browse/THRIFT-601
MG> The temporary workaround is "don't send random data to your Cassandra 
instance."

That wasn't random, I picked those letters very carefully! ;)

Thanks for explaining.  I looked at the Cassandra tickets but didn't see
this one reported already; I must have searched incorrectly.

Ted



out of memory error on malformed Thrift protocol

2009-11-13 Thread Ted Zlatanov
The sequence to trigger the bug:

1) telnet to port 9160

2) type:

s
s

(two letter+RET combinations)

This reliably generates the error below.  I did not debug further or try
to fix it because it seems to be a Thrift issue.

ERROR - Fatal exception in thread Thread[pool-1-thread-1,5,main]
java.lang.OutOfMemoryError: Java heap space
at 
org.apache.thrift.protocol.TBinaryProtocol.readStringBody(TBinaryProtocol.java:296)
at 
org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:203)
at 
org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:614)
at 
org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:619)

Thanks
Ted



Re: Cassandra access control

2009-11-13 Thread Ted Zlatanov
On Thu, 12 Nov 2009 16:24:11 -0800 "Coe, Robin"  wrote: 

CR> Why pass a map of credentials?  Why not follow the standard approach
CR> of opening the connection with the credentials, as in tr.open( uid,
CR> passwd )?  For now, that can be an overloaded method call that would
CR> leave the existing API as-is.

I would rather start with an unauthenticated connection, which is just
as standard (e.g. the IMAP protocol uses that approach, among many
others).  Otherwise you either end up with two open() methods or you
have extra code to handle null or failing credentials in the open()
method, and it has two failure modes.  Instead I would like to layer the
functionality: open() is for establishing the connection, login() is for
authenticating, setKeyspace() is for authorization.

UID and password are not the only credentials possible so I left it
open.  For example, we could pass tokens or anything else that fits in a
String.  I would rather have used Comparable but Thrift doesn't have
that as a basic type :)

Ted




Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 10:59:52 -0600 Ted Zlatanov  wrote: 

TZ> On Thu, 12 Nov 2009 10:49:59 -0600 Jonathan Ellis  
wrote: 
JE> On Thu, Nov 12, 2009 at 10:42 AM, Jonathan Mischo  
wrote:
>>> > Let's keep it simple.  Forcing multiple connections from a purely
>>> > hypothetical use case is a no-brainer tradeoff.  Connections are not
>>> > expensive.

>>> Even if we can do it sensibly? Connections aren't hugely expensive, but
>>> they're not free, either.

JE> I suppose, but if it requires sending a keyspace param w/ each call,
JE> then it's not sensible.  You waste far more overhead for that in the
JE> common case -- serializing, deserializing, checking that it's been
JE> authed -- than you gain from not having another connection in the
JE> uncommon one.

JE> I would be okay with being able to send a 2nd auth call to an existing
JE> connection to switch the "current" keyspace, similar to how rdbmses
JE> only have one active schema at a time.

TZ> How about:

TZ> login(Map credentials) throws 
CassandraAuthenticationSecurityException

TZ> setKeyspace(String keyspace) throws CassandraAuthorizationSecurityException

TZ> and then all the existing API calls won't have a Keyspace parameter as
TZ> previously discussed.  This works for everyone, I think, and separates
TZ> authentication from authorization nicely.

I created an issue:

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

and will post updates there as needed.  This is stage 1, meaning this is
the 0.5 work that will keep the old API.  Stage 2 will remove the
Keyspace parameters from the API so I'll put that work in a different
issue, linked to CASSANDRA-547.

Thanks
Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 10:49:59 -0600 Jonathan Ellis  wrote: 

JE> On Thu, Nov 12, 2009 at 10:42 AM, Jonathan Mischo  
wrote:
>> > Let's keep it simple.  Forcing multiple connections from a purely
>> > hypothetical use case is a no-brainer tradeoff.  Connections are not
>> > expensive.

>> Even if we can do it sensibly? Connections aren't hugely expensive, but
>> they're not free, either.

JE> I suppose, but if it requires sending a keyspace param w/ each call,
JE> then it's not sensible.  You waste far more overhead for that in the
JE> common case -- serializing, deserializing, checking that it's been
JE> authed -- than you gain from not having another connection in the
JE> uncommon one.

JE> I would be okay with being able to send a 2nd auth call to an existing
JE> connection to switch the "current" keyspace, similar to how rdbmses
JE> only have one active schema at a time.

How about:

login(Map credentials) throws 
CassandraAuthenticationSecurityException

setKeyspace(String keyspace) throws CassandraAuthorizationSecurityException

and then all the existing API calls won't have a Keyspace parameter as
previously discussed.  This works for everyone, I think, and separates
authentication from authorization nicely.

Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 08:23:51 -0800 "Coe, Robin"  wrote: 

CR> I agree.  Getting into LDAP will open a can of worms, especially if
CR> the plan is to support Active Directory.  There are a lot of RFCs on
CR> the subject of LDAP and Active Directory doesn't support them all.

CR> If LDAP is the plan, though, there needs to be support for ssl and
CR> tls, at a minimum.

Would you like to contribute or point me to a library that makes this
easier?  There's probably code in JAAS I can use, too, even if I'm not
using the whole library.  Anyhow, the simple file-based backend will
come first and perhaps that's all we need.

Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 10:11:27 -0600 Jonathan Ellis  wrote: 

JE> 2009/11/12 Ted Zlatanov :
>> It sounds like JAAS is a bad idea.  I'll use a modular auth system then,
>> with two simple implementations (XML file and LDAP) at first.  The XML
>> file will hold account passwords (one-way hashed) and authorizations.

JE> wouldn't it be simpler to just put the password hash in the keyspace 
definition?

JE> it's less enterprise but if you need something sophisticated you're
JE> probably going to use ldap anyway...

It's less secure and merges the auth backend configuration with the
Cassandra configuration.  I was hoping to keep them separate.

Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 10:23:21 -0600 Jonathan Mischo  
wrote: 

JM> The problem I see with this is that you can't have a single connection
JM> accessing multiple keyspaces at once.  I can think of some cases where
JM> having a single connection access and differentiate between two
JM> keyspaces would be advantageous, especially in certain kinds of
JM> reporting applications.  Otherwise, you force the creation and
JM> maintenance of multiple connection pools on the client side, which
JM> isn't as resource efficient.

JM> This goes back to the concept we were talking about on IRC yesterday
JM> where a single user may have access to more than one keyspace.  If you
JM> authenticate and the system authorizes access to multiple keyspaces,
JM> you should have access to them from the same connection, IMHO, since
JM> that's a pretty well established pattern.

It would also keep the API the same as it is now, just adding a login()
method.  I am mildly in favor of allowing multiple keyspaces but
Jonathan Ellis seems to prefer the other variant and he made some valid
points too.  I don't know if you guys want to vote or what, but I can
get started with the auth backend work regardless of the API direction.

Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 10:06:02 -0600 Jonathan Ellis  wrote: 

JE> 2009/11/12 Ted Zlatanov :
JE> The default should definitely be, "don't break people who don't need
JE> the new feature more than necessary."  So the default should be
JE> "accept any client to any keyspace."
>> 
>> Hmm, I thought we were going to limit access to a single keyspace upon
>> login.  You want to keep allowing multiple keyspaces?  That would leave
>> the existing API intact (only adding a login function) but requires an
>> extra authorization check every time a keyspace is given.  Do we expire
>> authorizations after a certain time?

JE> If this is going to 0.5 we should keep the existing API intact since
JE> we are very late in the 0.5 cycle (so, it's up to you if you need this
JE> in 0.5).  But ultimately we want to drop the keyspace args in which
JE> case the no-auth-configured behavior is that you still send an auth
JE> method call but the auth accepts whatever it is given.

I see.

So I'm adding a login() in 0.5 but keeping the Keyspace parameters
everywhere.  If the user has authenticated via login(), the Keyspace
logged in will be checked against the specified Keyspace (and exceptions
thrown if they don't match).  Otherwise, no check is done.  This keeps
the current API and behavior intact but adds the desired functionality.
The exception will point the user to the problem immediately.

For versions after 0.5, the current API calls with the Keyspace
parameter will be removed in favor of versions without it.  login() will
be required to specify the Keyspace regardless of whether authentication
is done or not.  The only expected security exception here comes from
login().  Once you're authorized, the grant doesn't expire.

If you're OK with all this I'll put together a full proposal in the Jira
ticket and start working on a patch to:

- add the login() method

- add an authentication+authorization interface called in the right
  places in 0.5

- implement that interface: provide a XML backend and a LDAP backend (no
  JAAS).  Also, a AllowAll backend will be provided.

- add the configuration file stanza to point to the
  authentication+authorization module to be used.  Make AllowAll the
  default auth backend there.

- document all the changes

Thanks
Ted



Cassandra access control (was: bandwidth limiting Cassandra's replication and access control)

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 07:28:29 -0800 "Coe, Robin"  wrote: 

CR> I'm not sure JAAS is the way to go when implementing a performant
CR> authentication/authorization service.  This is what threw me off in
CR> the first place.

OK.

CR> So, my concern is, and has been since this discussion started, is
CR> that Cassandra should not be performing this work.  I suggest that
CR> the Thrift connection is opened with credentials passed in, which
CR> Cassandra authenticates to.  Even this overhead is not something I
CR> would want to incur on every connection, so I would use a connection
CR> pool, with the connections pre-authenticated to a single account
CR> that's appropriate for my application.

It sounds like JAAS is a bad idea.  I'll use a modular auth system then,
with two simple implementations (XML file and LDAP) at first.  The XML
file will hold account passwords (one-way hashed) and authorizations.
The Apache Commons Configuration library can parse the XML file for us.
With LDAP, we'll use the property file for authorizations only.  We can
optimize the LDAP backend if needed.  Sounds good?

CR> Basically, the authentication on the Cassandra side needs to be
CR> lightning fast and performing LDAP lookups from Cassandra as each
CR> Thrift socket is opened will definitely impact performance.  

This depends on the application.  Mine, for example, use few sockets and
keep them open for a long time.  Keep in mind this only happens if the
user wants to login, too.  Unauthenticated access will not require any
lookups.

Ted



Cassandra access control (was: bandwidth limiting Cassandra's replication and access control)

2009-11-12 Thread Ted Zlatanov
On Wed, 11 Nov 2009 23:11:53 -0600 (CST) "Stu Hood"  
wrote: 

SH> Would you mind creating a ticket for this issue in JIRA? A lot of
SH> discussion has gone on, and a place to collect the design and
SH> feedback would be a good start.

I can, but I'd like to at least have a consensus with Jonathan on the
API before I go to issue tracking.  Also, do I need a Jira account and
permissions to create tickets?

Thanks
Ted



Re: Cassandra access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 09:12:56 -0600 Jonathan Ellis  wrote: 

JE> 2009/11/12 Ted Zlatanov :
>> On Wed, 11 Nov 2009 16:14:09 -0800 Anthony Molinaro 
>>  wrote:
>> 
AM> How will authentication work with non-java clients?  I don't think thrift
AM> itself has authentication built in, and it sounds like a java library is
AM> being proposed for the guts.  Will it still be possible to connect from
AM> a non-java client or will the thrift interface be deprecated?
>> 
>> The client will login with a Map of login tokens and get
>> an auth token (probably a String containing a UUID) back.  The token
>> will be valid for the duration of the client connection and will grant
>> access to a single keyspace.  Effectively, the token replaces the old
>> Keyspace argument in all Thrift API calls.

JE> I'd really prefer to just keep that around in a threadlocal.  There's
JE> no reason for a client to continue passing a token w/ each call that
JE> the server already knows.

OK.

>> I am thinking of allowing dual operation where if you pass a keyspace
>> name without login, it works on servers that don't have authentication
>> enabled.

JE> The default should definitely be, "don't break people who don't need
JE> the new feature more than necessary."  So the default should be
JE> "accept any client to any keyspace."

Hmm, I thought we were going to limit access to a single keyspace upon
login.  You want to keep allowing multiple keyspaces?  That would leave
the existing API intact (only adding a login function) but requires an
extra authorization check every time a keyspace is given.  Do we expire
authorizations after a certain time?

Ted



Cassandra access control (was: bandwidth limiting Cassandra's replication and access control)

2009-11-12 Thread Ted Zlatanov
On Wed, 11 Nov 2009 14:59:04 -0800 "Coe, Robin"  wrote: 

CR> Java's policy manager controls access to environment variables and
CR> code execution.  All a JAAS service provides is a hook to pass a
CR> user's principal to the security manager.  So, the only
CR> authorization you can provide at that level is code executed by the
CR> Principal.  This doesn't include access via the Thrift API, since
CR> that code is owned by the user who started the Cassandra service.

This is my understanding from the JAAS docs and tutorials: with the JNDI
JAAS module, you can pass a user name and a password as properties
instead of relying on the current user.  Then you can grant access to
various resources, not just the default set (file/process resources).
This is what I was hoping to use.  The Kerberos and Unix JAAS modules
are definitely not useful because they rely on the current user, yes.

If JAAS is not going to work, I'll just do an LDAP backend directly.  It
would cover over 95% of the setups out there and it's not too hard to
implement with JNDI calls.  I hope it's not necessary to go that route.
WDYT?

Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-12 Thread Ted Zlatanov
On Wed, 11 Nov 2009 10:14:58 -0600 Eric Evans  wrote: 

EE> On Tue, 2009-11-10 at 16:25 -0600, Ted Zlatanov wrote:
>> (BTW, I use Eclipse for Java development, is there a way to run the
>> Ant tasks automatically to rebuild the generated source if necessary?
>> It works fine otherwise.) 

EE> Yes. Go to Project->Properties->Builders, click "New", and add an Ant
EE> Builder (the defaults should be OK). You'll probably want to disable
EE> Java Builder while you're there.

Thanks.  I actually like the Java builder too (I have automatic builds
turned on) but the Ant builder was really simple to set up and use.

Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-12 Thread Ted Zlatanov
On Thu, 12 Nov 2009 12:40:05 +1100 Ian Holsman  wrote: 

IH> most places i've seen don't use DB auth anywhere. there is a common
IH> login, stored in a property file, sometimes stored in a internally- 
IH> world-readable SVN repo.

In my current industry (financials) this is not acceptable.  It puts
money and jobs at risk to open access this way.

IH> they usually use network ACLs to restrict access to good hosts. (jump
IH> hosts). network ACLs have been tested for decades and they work.
IH> implementing your own auth is just asking for problems. It's too hard
IH> to do properly, and will probably never work well with the enterprises
IH> existing auth systems.

Layers of security are always a good idea (any firewall is just a part
of good security design, and by itself only increases complacency).  I
should mention I've been a sysadmin and network admin for many years
besides doing programming.

No one is suggesting to implement our own authentication.  We're going
to use existing mechanisms, namely what JAAS supports (LDAP, NIS,
etc.).  We're creating a specific authorization mechanism because it
makes sense for Cassandra, but we're again using JAAS to do that.

IH> If you have sensitive data being stored, ENCRYPT it, or use a 1-way
IH> hash instead of storing it.  Ideally with a user-supplied key which
IH> is not stored anywhere on disk.

This is not feasible in many cases.  Encryption is slow and very hard to
implement properly.  One-way hashes lose the original content,
obviously.  User-supplied keys require interactivity at least at some
point, which is annoying and makes reliable operation harder to
achieve.  Fast access to the data is very important and my proposal
(initial login followed by an auth token passed around) is a decent
solution to these concerns.

IH> sadly DBA's are people too, and it is pathetically easy for them to
IH> get all the data from a DB-dump.

Securing backups is, fortunately, much easier to address on the server
side because it deals with static data.

Ted



Cassandra access control (was: bandwidth limiting Cassandra's replication and access control)

2009-11-12 Thread Ted Zlatanov
On Wed, 11 Nov 2009 16:14:09 -0800 Anthony Molinaro 
 wrote: 

AM> How will authentication work with non-java clients?  I don't think thrift
AM> itself has authentication built in, and it sounds like a java library is
AM> being proposed for the guts.  Will it still be possible to connect from
AM> a non-java client or will the thrift interface be deprecated?

The client will login with a Map of login tokens and get
an auth token (probably a String containing a UUID) back.  The token
will be valid for the duration of the client connection and will grant
access to a single keyspace.  Effectively, the token replaces the old
Keyspace argument in all Thrift API calls.

I am thinking of allowing dual operation where if you pass a keyspace
name without login, it works on servers that don't have authentication
enabled.  If the keyspace name is really an auth token (say,
"a...@uuid-goes-here") then we look up that auth token and proceed
accordingly.  So the old Thrift clients will work with new Cassandra
servers as long as authentication is disabled.

Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-12 Thread Ted Zlatanov
On Wed, 11 Nov 2009 15:48:41 -0700 Joe Stump  wrote: 

JS> On Nov 11, 2009, at 3:29 PM, Alexander Vushkan wrote:

>> ...but authentication support would be nice to have...

JS> I'll continue to object to this. If you're considering running
JS> Cassandra (or MySQL or Reddis or Memcache or MemcacheDB or ...) on an
JS> open network Ur Doin' It Wrong. This is what VPN's were created for.
JS> Nobody in their right mind runs stuff like this, in production,
JS> exposed to the outside world.

Huh?  So we're going to design insecure software by default because we'd
be idiots to expose insecure software?  That's circular reasoning.

Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-11 Thread Ted Zlatanov
On Wed, 11 Nov 2009 07:40:00 -0800 "Coe, Robin"  wrote: 

CR> Just going to chime in here, because I have experience writing apps
CR> that use JAAS and JNDI to authenticate against LDAP and JDBC
CR> services.  However, I only just started looking at Cassandra this
CR> week, so I'm not certain of the premise behind controlling access to
CR> the Cassandra service.

CR> IMO, auth services should be left to the application layer that
CR> interfaces to Cassandra and not built into Cassandra.  In the
CR> tutorial snippet included below, the access being granted is at the
CR> codebase level, not the transaction level.  Since users of Cassandra
CR> will generally be fronted by a service layer, the java security
CR> manager isn’t going to suffice.  What this snippet could do, though,
CR> and may be the rationale for the request, is to ensure that
CR> unauthorized users cannot instantiate a new Cassandra server.
CR> However, if a user has physical access to the machine on which
CR> Cassandra is installed, they could easily bypass that layer of
CR> security.

CR> So, I guess I'm wondering whether this discussion pertains to
CR> application-layer security, i.e., permission to execute Thrift
CR> transactions, or Cassandra service security?  Or is it strictly a
CR> utility function, to create a map of users to specific Keyspaces, to
CR> simplify the Thrift API?

(note followups to the devel list)

I mentioned I didn't know JAAS so I appreciate any help you can give.
Specifically, I don't know yet what is the difference between the
codebase level and the transaction level in JAAS terms.  Can you
explain?

I am interested in controlling the Thrift client API, not the Gossip
replication service.  The authenticating clients will not have physical
access to the machine and all the authentication tokens will have to be
passed over a Thrift login call. How would you use JAAS+JNDI to control
that?  The access point is CassandraServer.java as Jonathan mentioned.

Ted



Re: [ANNOUNCE] Cassandra.gem 0.5

2009-11-11 Thread Ted Zlatanov
On Thu, 20 Aug 2009 14:37:55 -0700 Evan Weaver  wrote: 

EW> The Cassandra client gem 0.5 for Ruby is released!
EW> Highlights since the last ANNOUNCE:
EW>   - gem name changed from cassandra_client to cassandra
EW>   - bin/cassandra_helper script, to build and start the server for you
EW>   - working temporal comparators, including RFC-compliant UUID type
EW>   - many API changes, for usability
EW>   - complete documentation

EW> This is the first version I would be comfortable using in production,
EW> which is saying something.

EW> To install, run: sudo gem install cassandra

EW> Documentation is here: http://blog.evanweaver.com/files/doc/fauna/cassandra/

Thanks for this package.  I tried it without much Ruby experience on
Ubuntu.  The 'rubygems' package was required.  I also needed the
'libopenssl-ruby' package, but there was no way to tell it was required
during the installation.  After install, cassandra_helper complained
that net/https was missing and that was the only hint I had that
libopenssl-ruby needed to be installed.

Apparently all the binaries get installed into either ~/.gem or
/var/lib/gems, silently, depending on whether root or a user is running.
This is a bit annoying (not your fault, of course) as I had to do

CASSANDRA_INCLUDE=/home/tzz/workspace/Cassandra 
PATH=/home/tzz/.gem/ruby/1.8/bin:/var/lib/gems/1.8/bin:${PATH} cassandra_helper 

just to run the helper.  This maybe should be mentioned in the docs;
right now they just say to use bin/cassandra_helper.

The build_cassandra command worked great.  Starting cassandra, however,
gave me:

CASSANDRA_INCLUDE=/home/tzz/workspace/Cassandra 
PATH=/home/tzz/.gem/ruby/1.8/bin:/var/lib/gems/1.8/bin:${PATH} cassandra_helper 
cassandra
(in /home/tzz/.gem/ruby/1.8/gems/cassandra-0.5.6.2)
mkdir -p /home/tzz/cassandra/test
[: 46: -o: unexpected operator
Usage: java [-options] class [args...]
   (to execute a class)
   or  java [-options] -jar jarfile [args...]
   (to execute a jar file)

... (java help output cut) ...

I'm using zsh, perhaps that's the issue.  I didn't debug it further
right now but will look further if it's not obvious to you where the
problem is.

Thanks
Ted



Re: Digg's Python Cassandra Client: Lazyboy

2009-11-11 Thread Ted Zlatanov
On Wed, 22 Jul 2009 21:42:31 -0700 Chris Goffinet  wrote: 

CG> Just wanted to let everyone know we released our client tonight at the
CG> Cassandra hackathon, its for Python 2.5x (haven't tested 2.4). It's up
CG> on Github:

CG> http://github.com/digg/lazyboy/tree/master

Hi Chris,

I git-cloned the client but it's really just a library, there's no REPL
wrapper.  Would it be hard to add that?  I don't know Python well enough
to say, but it would be nice to interact with Cassandra from Python
similarly to the cassandra-cli program.

Also, there's no documentation on installing it just for personal use,
and on the functionality of the library.  The examples are a start but
real docs would be nice.

Thanks
Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-11 Thread Ted Zlatanov
On Tue, 10 Nov 2009 17:09:44 -0600 Jonathan Ellis  wrote: 

JE> 2009/11/10 Ted Zlatanov :
>> I see all the methods implementing the server interface in
>> org.apache.cassandra.service.CassandraServer.  Is that where the
>> authentication should happen?  Should I use JAAS (LDAP+PAM+other
>> backends) or is there something else preferred?

JE> Right, CassandraServer is the first non-generated stop for Thrift 
connections.

JE> JAAS looks like the standard java way to do this sort of thing, which
JE> makes me a little suspicious. :)  

I looked for other solutions, but nothing was as convenient.  The docs
for the NIS auth backend show incorrect sample data (the LDAP sample is
copied) which makes me a lot suspicious.  It's not too enterprisey, though.

JE> But if you're already familiar with that, let's run with it.

I will learn it, I don't know it well yet.  Bear with me; discussion
follows.  Should we move this to the devel list, BTW?

>> Does libThrift allow persistent state per client somehow or do I have to
>> implement my own security tokens?  I don't see a way to track the client
>> in any of the Cassandra services currently so I suspect libThrift is
>> stateless on its own.

JE> Yes.  Since we're using the thrift threaded server, setting up a
JE> threadlocal from a login method and leaving the other methods alone
JE> would be my preferred approach.

Is it OK to keep the local auth info as a field in the CassandraServer
instance or is another place better?

JE> Ultimately what I'd like is to have a client authenticate to a
JE> specific keyspace.  Then we'd drop the keyspace argument from the
JE> other methods.

It looks like JAAS lets us use a jaas.conf file to specify the
authentication backend.  The application just needs to specify the
java.security.auth.login.config property as the full name of the
jaas.conf file.

The JNDI login module (LDAP and NIS) supports these properties,
according to the docs at 
http://java.sun.com/javase/6/docs/jre/api/security/jaas/spec/index.html:

javax.security.auth.login.name
javax.security.auth.login.password

before login() is called.  The other JAAS modules don't support that
(AFAICT they only work on the current user) so we'd only be able to
authenticate based on NIS or LDAP, or other JNDI providers.  This should
support at least Active Directory and most Unix shops.

As far as authorization after authentication, JAAS supports config files
like this (from the tutorial):

grant codebase "file:./SampleAction.jar",
Principal javax.security.auth.kerberos.KerberosPrincipal 
"your_user_n...@your_realm"  {

   permission java.util.PropertyPermission "java.home", "read";
   permission java.util.PropertyPermission "user.home", "read";
   permission java.io.FilePermission "foo.txt", "read";
};

The Principal is the user ID.

We should be able to use anything that extends java.security.Permission
here; I think you're suggesting a KeyspacePermission but we should also
have a ClusterPermission.  I think we should allow wildcards in the
resource name.

This seems like a reasonable way to configure things and it does not
require any changes to the existing configuration files.  I like that;
separating security policies from the other configuration is always a
good thing.

See the tutorials for the details, starting at:

http://java.sun.com/javase/technologies/security/index.jsp
http://java.sun.com/javase/6/docs/technotes/guides/security/index.html

The actual auth method is pretty simple then.  Passing the password in a
non-Kerberos environment is risky but I am not sure of the right way to
encrypt the channel just for the password.  Encrypting just the login
info requires some thought as well.  Maybe someone has done this before?
For now the auth can just be

public void authenticateUser(Map credentials, String cluster, 
String keyspace) throws ???

where the credentials are just the 'user' and 'password' keys in that map.

If there are no objections I'll put together a patch.

Thanks
Ted



Re: bandwidth limiting Cassandra's replication and access control

2009-11-10 Thread Ted Zlatanov
On Tue, 10 Nov 2009 13:53:58 -0600 Jonathan Ellis  wrote: 

JE> 2009/11/10 Ted Zlatanov :
>> I also would really like a way to limit access to the Thrift interface
>> with at least some rudimentary username/password combination.  I don't
>> see a way to do that currently.

JE> Yeah, lots of people want that, somebody should code it up. :)

(BTW, I use Eclipse for Java development, is there a way to run the Ant
tasks automatically to rebuild the generated source if necessary?  It
works fine otherwise.)

I see all the methods implementing the server interface in
org.apache.cassandra.service.CassandraServer.  Is that where the
authentication should happen?  Should I use JAAS (LDAP+PAM+other
backends) or is there something else preferred?  

Does libThrift allow persistent state per client somehow or do I have to
implement my own security tokens?  I don't see a way to track the client
in any of the Cassandra services currently so I suspect libThrift is
stateless on its own.

Thanks
Ted



bandwidth limiting Cassandra's replication and access control

2009-11-10 Thread Ted Zlatanov
I'm evaluating Cassandra as a storage mechanism for monitoring data:
machine and process status reports, inventory, etc.

One of my concerns is bandwidth usage; I don't want Cassandra
replication traffic swamping more important traffic.  I want to know if
there's a way to limit the bandwidth usage within Cassandra or if I have
to resort to network-level QoS rules instead.  QoS rules get a little
complicated when you try to distinguish traffic within the data center
from traffic to the outside; I was hoping to avoid that.

I also would really like a way to limit access to the Thrift interface
with at least some rudimentary username/password combination.  I don't
see a way to do that currently.

Thanks for any help
Ted