As a side note, be aware that running with DEBUG logging enabled can make
your cluster run a full order of magnitude slower.
On Mon, May 9, 2011 at 6:54 PM, Suan Aik Yeo wrote:
> Ah, must be the status check that I set up. Thanks!
>
>
> On Mon, May 9, 2011 at 7:42 PM, Tyler Hobbs wrote:
>
>> It
With a very small amount of memory, the Cassandra process may be getting
killed by the Linux OOM killer, which should result in a log message to the
kernel logs. See
http://stackoverflow.com/questions/624857/finding-which-process-was-killed-by-linux-oom-killerto
locate the error if it exists.
On F
What version of Cassandra were you testing with?
On Wed, Jan 5, 2011 at 6:11 AM, David Boxenhorn wrote:
> The CLI sometimes gets only 100 results (even though there are more) - and
> sometimes gets all the results, even when there are more than 100!
>
> What is going on here? Is there some logic
Raj: the super column indexing is a longstanding issue that we've been
considering recently, and would like to fix. See
https://issues.apache.org/jira/browse/CASSANDRA-674
On Fri, Jan 7, 2011 at 10:53 PM, Tyler Hobbs wrote:
> Not that I'm aware of. There are several other decent alternatives to
> And I notice in 0.7 roadmap there is a feature called "vector clock
support"
The orginal plan was to implement vector clocks for Cassandra, but
Cassandra's data model actually provides at alternative solution that we'd
like to start recommending. If you know that you will be experiencing
unavoida
As mentioned in the article you linked, index creation happens
asynchronously: when you perform the schema update call to add an index, the
index starts building in the background, and will not be completely valid
until it finishes building. I believe there is a JMX call to check the
status of this
Opened https://issues.apache.org/jira/browse/CASSANDRA-2006 with the
solution we had suggested on the MultiTenant wiki page.
On Tue, Jan 18, 2011 at 11:56 PM, David Boxenhorn wrote:
> I think tuning of Cassandra is overly complex, and even with a single
> tenant you can run into problems with to
Also note that an improved and compressible file format has been in the
works for a while now.
https://issues.apache.org/jira/browse/CASSANDRA-674
I am endlessly optimistic that it will make it into the 'next' version; in
particular, the current hope is 0.8
On Jan 20, 2011 6:34 AM, "Terje Marthi
When the destination node fails to open the streamed SSTable, we assume it
was corrupted during transfer, and retry the stream. Independent of the
exception posted above, it is a problem that the failed transfers were not
cleaned up.
How many of the data files are marked as -tmp-?
On Jan 27, 2011
Our intention was that if you wanted to add another permission like "update"
(a subset of "write") then you would return it from the method as part of
the EnumSet for that resource. I would see how much trouble it
would be to add a new Permission value for "update".
Note that Cassandra itself does
Not only does the type need to make sense, but it also needs to sort in
exactly the same order as the previous type did... in which case there would
be no reason to change it?
We should probably just say "no, you cannot do this", and explicitly prevent
it.
On Wed, Feb 2, 2011 at 3:14 PM, Jonathan
The expired columns were converted into tombstones, which will live for the
GC timeout. The "empty" row will be cleaned up when those tombstones are
removed.
Returning the empty row is unfortunate... we'd love to find a more
appropriate solution that might not involve endless scanning.
See
http:/
Alexander:
The secondary indexes in 0.7.0 (type KEYS) are stored internally in a column
family, and are kept synchronized with the base data via locking on a local
node, meaning they are always consistent on the local node. Eventual
consistency still applies between nodes, but a returned result wi
> Does it also mean that the whole row will be deserialized when a query
comes
> just for one column?
No, it does not mean that: at most column_index_size_in_kb will be read to
read a single column, independent of where that column is in the row.
On the other hand, with the row cache enabled, it i
_But_, vote for https://issues.apache.org/jira/browse/CASSANDRA-1472 if
you'd like to be able to perform this type of query easily*. Binned bitmap
indexes can perform compound range queries extremely quickly.
* Assuming that your data isn't extremely volatile, in which case those
indexes are not t
I would like to continue to support super columns, but to slowly convert
them into "compound column names", since that is really all they really are.
On Thu, Feb 10, 2011 at 10:16 AM, Frank LoVecchio wrote:
> I've found super column families quite useful when using
> RandomOrderedPartioner on a l
But, the reason that it isn't safe to say that we are a strongly consistent
store is that if 2 of your 3 replicas were to die and come back with no
data, QUORUM might return the wrong result.
A requirement of a strongly consistent store is that replicas cannot begin
answering queries until they ar
I expect that this problem was due to
https://issues.apache.org/jira/browse/CASSANDRA-2216 : I'll make noise to
try and get it released soon as 0.7.3
On Tue, Feb 22, 2011 at 5:41 AM, David Boxenhorn wrote:
> Thanks, Shimi. I'll keep you posted if we make progress. Riptano is working
> on this pr
In practice, local secondary indexes scale to {RF * the limit of a single
machine} for -low cardinality- values (ex: users living in a certain state)
since the first node is likely to be able to answer your question. This also
means they are good for performing filtering for analytics.
On the othe
The comment in the example config file next to that setting explains it more
fully, but something like 16 * number of drives is a reasonable setting for
readers. Writers should be a multiple of the number of cores.
On Thu, Mar 17, 2011 at 1:09 PM, buddhasystem wrote:
> Hello, in the instructions
The storage proxy latencies are the primary metric: in particular, the
latency histograms show the distribution of query times.
On Fri, Apr 8, 2011 at 5:27 PM, mcasandra wrote:
> What are the key things to monitor while running a stress test? There is
> tons
> of details in nodetoll tpstats/net
If an SSTable contains an update for a row (row, not just column), we need
to read from it. See #1608 for some of the ideas that have been floated on
how to improve this situation: the core ones are 1. partitioning local data
so that the the number of files involved in a read is smaller, 2. adding
Sorry, I meant to say #2319:
https://issues.apache.org/jira/browse/CASSANDRA-2319
On Sat, Apr 9, 2011 at 6:04 PM, Stu Hood wrote:
> If an SSTable contains an update for a row (row, not just column), we need
> to read from it. See #1608 for some of the ideas that have been floated on
&g
y the existing implementation, but
constant updates to rows will lower your performance until a solution to
#1608 is available.
On Sat, Apr 9, 2011 at 7:22 PM, Stu Hood wrote:
> Sorry, I meant to say #2319:
> https://issues.apache.org/jira/browse/CASSANDRA-2319
>
>
> On Sat, Apr 9,
The row index is an index of the columns stored in a particular row: it is
only written when a row gets larger than column_index_size_in_kb (see your
config file). The sstable index is currently an index of the keys stored in
an sstable, but #2319 proposes to merge the sstable and row indexes.
On
Readonly Compactions are used to hash column families for
http://wiki.apache.org/cassandra/ArchitectureAntiEntropy . Roger's link refers
to anticompaction specifically.
-Original Message-
From: "Weijun Li"
Sent: Wednesday, May 5, 2010 11:29am
To: user@cassandra.apache.org
Subject: Re: A
Hey Ed,
I've been working on a similar approach for arbitarily nested/compound column
names in #998. See:
http://github.com/stuhood/cassandra/blob/998/src/java/org/apache/cassandra/db/ColumnKey.java
The goal is to provide native support and potentially (in the very long term),
API support for
Ian: I think that as get_range_slice gets faster, the approach that Mark was
heading toward may be considerably more efficient than reading the old value in
the OutputFormat.
Mark: Reading all of the data you want to update out of Cassandra using the
InputFormat, merging it with (tagged) new da
ranularities (i.e. each entry
in the SortedMap> map might correspond to a single input
row from the other input stream).
I'll put something on the wiki if I can make it work...
Cheers
Mark
On Fri, May 7, 2010 at 8:26 AM, Stu Hood wrote:
> Ian: I think that as get_range_slice gets fa
Your IPartitioner implementation decides how the row keys are sorted: see
http://wiki.apache.org/cassandra/StorageConfiguration#Partitioner . You need to
be using one of the OrderPreservingPartitioners if you'd like a reasonable
order for the keys.
-Original Message-
From: "AJ Chen"
Se
You're right, it should be private. But... I don't think it is worth opening a
ticket for.
-Original Message-
From: "Eben Hewitt"
Sent: Saturday, May 8, 2010 11:37am
To: user@cassandra.apache.org
Subject: AntiEntropyService Singleton
Hi Everyone
Thanks for your all of your terrific work
I think that it is 100% ideal: it's what I've been working on implementing in
#674, #847 and #998. I'm hoping to post a large patchset and docs this week,
and I'm aiming to get it committed for 0.8.
The work I've been doing doesn't touch the user interface: it only deals with
the internal chang
The Hadoop integration (as demonstrated by contrib/word_count) is locality
aware: it begins by querying Cassandra to generate locality aware splits, and
when the hostnames match up between the Hadoop and Cassandra clusters, the data
can be mapped locally.
-Original Message-
From: "Maxim
A Cassandra OutputFormat was recently contributed, but I haven't had a chance
to review it. Any feedback you can give would be awesome:
https://issues.apache.org/jira/browse/CASSANDRA-1101
Thanks,
Stu
-Original Message-
From: "Jeff Zhang"
Sent: Tuesday, May 18, 2010 4:39am
To: user@cass
Also, when you are testing trunk, please remember to read NEWS.txt, as things
change frequently.
-Original Message-
From: "Nathan McCall"
Sent: Tuesday, May 18, 2010 11:36am
To: user@cassandra.apache.org
Subject: Re: cassandra.yaml not picked up?
I came across this the other day as well
50% of 0 will be rounded up to 1.
-Original Message-
From: "Ran Tavory"
Sent: Monday, May 24, 2010 12:34am
To: user@cassandra.apache.org
Subject: Key cache capacity: 1 when using KeysCached="50%"
I've noticed that when defining KeysCached="50%" (or KeysCached="100%" and I
didn't test ot
See https://issues.apache.org/jira/browse/CASSANDRA-579 for some background
here: I was just about to start working on this one, but it won't make it in
until 0.7.
-Original Message-
From: "Sean Bridges"
Sent: Wednesday, May 26, 2010 11:50am
To: user@cassandra.apache.org
Subject: using
Did you watch in the logs to confirm that repair had actually finished? The
`nodetool repair` call is not blocking before 0.6.3 (unreleased): see
https://issues.apache.org/jira/browse/CASSANDRA-1090
-Original Message-
From: "James Golick"
Sent: Sunday, May 30, 2010 3:43am
To: cassandra-
ool repair?
It may not have actually finished at that point. Though, according to JMX,
both compactions of each CF had completed, so I assumed it was done.
On Mon, May 31, 2010 at 11:29 AM, Stu Hood wrote:
> Did you watch in the logs to confirm that repair had actually finished? The
>
A "major" compaction is any compaction that sees all of the sstables for a
column family. In the context of the method you edited, that means that all of
the SSTables fall into a single bucket, and can be compacted together.
-Original Message-
From: "casablinca126.com"
Sent: Friday, Jun
Hey Dave,
This won't work out of the box, but it should be relatively easy to fix.
Implementing a TextColumnFamilyInputFormat that wraps ColumnFamilyInputFormat
to convert the datastructures it outputs to JSON/TSV/CSV.
If you have time to work on this, there is an open ticket:
https://issues.a
See https://issues.apache.org/jira/browse/CASSANDRA-981
-Original Message-
From: "Benjamin Black"
Sent: Friday, June 18, 2010 12:32pm
To: user@cassandra.apache.org
Subject: Re: Failover and slow nodes
Would be interesting to have a snitch that manipulated responses for
read nodes based o
Could we conditionally use an MD5 request only if a node was in a different
zone/datacenter according to the replication strategy? Presumably the bandwidth
usage within a datacenter isn't a concern as much as latency.
-Original Message-
From: "Mason Hale"
Sent: Wednesday, July 7, 2010 1
Cassandra has a very high constant per-row overhead at the moment of around 40
bytes. Additionally, there is around 12 bytes of overhead per column. Finally,
column names are repeated for each row.
CASSANDRA-674 and CASSANDRA-1207 will help with these overheads, but they will
not be fixed until
The Thrift server is embedded in Cassandra, and starts by default. Look for
references to Thrift on: http://wiki.apache.org/cassandra/GettingStarted
Thanks,
Stu
-Original Message-
From: "S Ahmed"
Sent: Monday, July 12, 2010 3:43pm
To: user@cassandra.apache.org
Subject: server needs thri
d, why does the installation guide say to build and make it then?
http://github.com/ericflo/twissandra
<http://github.com/ericflo/twissandra>twissandar is for 0.6.1 is that why?
i.e. it was embedded in a later version?
On Mon, Jul 12, 2010 at 4:46 PM, Stu Hood wrote:
> The Thrift server is em
Hello out there,
If you are running Cassandra 0.6.*, and are using Cassandra's authentication
(IAuthenticator/SimpleAuthenticator), I'd love to hear about it!
Thanks,
Stu Hood
@stuhood
Architecture Software Developer
Rackspace Hosting
Hey Oren,
The Cloud Servers REST API returns a "hostId" for each server that indicates
which physical host you are on: I'm not sure if you can see it from the control
panel, but a quick curl session should get you the answer.
Thanks,
Stu
-Original Message-
From: "Oren Benjamin"
Sent:
ize
before each run.
If anyone has explanations or suggestions, they would be quite welcome. This is
surprising to say the least.
Cheers,
Dave
On Jul 19, 2010, at 11:42 AM, Stu Hood wrote:
> Hey Oren,
>
> The Cloud Servers REST API returns a "hostId" for each server that
Rackspace Cloud
> How many physical client machines are running stress.py?
One with 50 threads; it is remote from the cluster but within the same
DC in both cases. I also run the test with multiple clients and saw
similar results when summing the reqs/sec.
On Mon, Jul 19, 2010 at 1:22 PM, St
If you put 25 processes on each of the 2 machines, all you are testing is how
fast 50 processes can hit Cassandra... the point of using more machines is that
you can use more processes.
Presumably, for a single machine, there is some limit (K) to the number of
processes that will give you addit
Did you copy the data directories from one node to the others?
http://wiki.apache.org/cassandra/FAQ#cloned
-Original Message-
From: "Dathan Pattishall"
Sent: Tuesday, July 20, 2010 6:09pm
To: user@cassandra.apache.org
Subject: how come some nodes will drop nodes from the ring and not oth
Can you determine approximately what revisions you were running before and
after?
-Original Message-
From: "Arya Goudarzi"
Sent: Thursday, July 29, 2010 4:42pm
To: user@cassandra.apache.org
Subject: Avro Runtime Exception Bad Index
Just wanted to toss this out there in case if this is a
traffic at the time update
happens.
- Original Message -----
From: "Stu Hood"
To: user@cassandra.apache.org
Sent: Thursday, July 29, 2010 2:52:48 PM
Subject: RE: Avro Runtime Exception Bad Index
Can you determine approximately what revisions you were running before and
afte
That error is coming from the frontend: the jars must also be on the local
classpath. Take a look at how contrib/pig/bin/pig_cassandra sets up
$PIG_CLASSPATH.
-Original Message-
From: "Christian Decker"
Sent: Friday, August 13, 2010 11:30am
To: user@cassandra.apache.org
Subject: Cassand
r.java:510)
at java.lang.Thread.dispatchUncaughtException(Thread.java:1845)
Any idea? Thanks again for your fast answer :)
On Fri, Aug 13, 2010 at 6:55 PM, Stu Hood wrote:
> That error is coming from the frontend: the jars must also be on the local
> classpath. Take a look at how contrib/
nk/0.7-beta2 requires properties
to be set by your startup script.
-Original Message-
From: "Stu Hood"
Sent: Friday, August 13, 2010 2:31pm
To: user@cassandra.apache.org
Subject: Re: Cassandra and Pig
Hmm, the example code there may not have been run in distributed mode recently,
Needing to manually copy the jars to all of the nodes would mean that you
aren't applying the Pig 'register ;' command properly.
-Original Message-
From: "Christian Decker"
Sent: Wednesday, August 18, 2010 7:08am
To: user@cassandra.apache.org
Subject: Re: Cassandra and Pig
I got one ste
Hey Aaron,
We are thinking a lot about multi-tenancy, but features to support multiple
tenants on a cluster are only beginning to make their way into Cassandra. See
http://wiki.apache.org/cassandra/MultiTenant for a short listing of features
that are being considered (including a mention of mem
See http://wiki.apache.org/cassandra/FAQ#no_keyspaces , or the Upgrading
section in NEWS.txt.
-Original Message-
From: "Frank LoVecchio"
Sent: Sunday, August 22, 2010 11:10pm
To: user@cassandra.apache.org
Subject: No Keyspace found after fresh .7 install
I now have 2 running nodes : one
JNA is _not_ necessary to use Cassandra, but the server can perform some
operations more efficiently if JNA is in place.
Not sure what is causing the error you are seeing in the CLI though: those
statements appear to be valid.
-Original Message-
From: "Mike Peters"
Sent: Thursday, Sept
Cassandra supports the recommended approach from:
http://www.percona.com/ppc2009/PPC2009_mysql_pagination.pdf
For large numbers of items, skip + limit is extremely inefficent.
-Original Message-
From: "Michal Augustýn"
Sent: Sunday, September 5, 2010 5:39am
To: user@cassandra.apache.org
Minor compactions will often be able to perform this garbage collection as well
in 0.6.6 and 0.7.0 due to a great optimization implemented by Sylvain:
https://issues.apache.org/jira/browse/CASSANDRA-1074
-Original Message-
From: "Aaron Morton"
Sent: Wednesday, September 22, 2010 7:47pm
Take a look at your particular implementation of
org.apache.cassandra.dht.IPartitioner: each partitioner: creates tokens in a
different way, but all of them are straightforward.
- Reply message -
From: "Dwight Smith"
Date: Fri, Sep 24, 2010 1:07 pm
Subject: How can the cassandra token o
Take a look at the get_indexed_slices method in the 0.7.0-beta Thrift interface.
-Original Message-
From: "Christian Decker"
Sent: Thursday, September 30, 2010 4:38am
To: user@cassandra.apache.org
Subject: Re: LongType from user input
I just read through the tickets on Jira, and it appea
Coool. Would you mind opening an Avro issue for that, or should I?
-Original Message-
From: "Gabor Torok"
Sent: Thursday, September 30, 2010 2:36pm
To: "user@cassandra.apache.org"
Subject: RE: avro + cassandra + ruby
The ruby code creates a new http connection for each call to transceiv
Hey JT,
I believe this issue should be fixed by CASSANDRA-1571... if you're able to
test that patch, it would be very helpful.
Thanks,
Stu
-Original Message-
From: "J T"
Sent: Tuesday, October 5, 2010 9:50pm
To: cassandra-u...@incubator.apache.org
Subject: Null Pointer Exception / Seco
> Specifically I'm wondering if I could create a byte representation of the Long
> that would also be lexicographically ordered.
This is probably what you want to do, combined with the ByteOrderedPartitioner
in 0.7
-Original Message-
From: "Eric Czech"
Sent: Friday, October 22, 2010 7:05
While the "adding virtual tokens/nodes to Cassandra" discussion is a good one,
there are a few factors that might delay (or remove?) the necessity of adding
that complexity:
* In Cassandra 0.7, removing load from a node is fairly cheap: a bounded number
of reads are used to determine which port
What column comparator/type are you using? Remember that if you are using
BytesType/UTF8Type, columns will be sorted lexicographically.
-Original Message-
From: "zangds"
Sent: Friday, November 5, 2010 8:53am
To: "user"
Subject: how does get_range_slices work?
Hi,
I have a question ab
The actually MerkleTree itself is at org.apache.cassandra.utils.MerkleTree: it
has a reasonable number of tests in the MerkleTreeTest class, and Cassandra
uses a tree to store the hashes of a ColumnFamily in o.a.c.d.CompactionManager
via a o.a.c.s.AntiEntropyService "Validator".
-Original
Hey Aditya,
Would you mind attaching that last hundred few lines from before the exception
from the server log to this ticket:
https://issues.apache.org/jira/browse/CASSANDRA-1724 ?
Thanks,
Stu
-Original Message-
From: "Jeremy Hanna"
Sent: Wednesday, November 10, 2010 11:40am
To: user
At first glance, this appeared to be a very egregious bug, but the effect is
actually minimal: since the size of the buffer is deterministic based on the
size of the data, you will have equal amounts of excess/junk data for equal
rows. Combined with the fact that 0.6 doesn't reuse these buffers,
Is the server logging anything during the failed authentication?
On Fri, Nov 12, 2010 at 8:07 PM, Alaa Zubaidi wrote:
> using SimpleAuthenticator is not working with me in beta 3
>
> I am doing the following:
>
> · In Cassandra.yaml Set
> authenticator: org.apache.cassandra.auth.Simple
All write patterns should provide the same performance with Cassandra, since
all writes to disk occur sequentially. The only variance might be in the data
structure used for the Memtable (a concurrent skip list), but I expect that it
is quite stable.
See http://www.mikeperham.com/2010/03/13/cas
If you have debug logs from the run, would you mind opening a JIRA describing
the problem?
-Original Message-
From: "Todd Burruss"
Sent: Sunday, March 21, 2010 1:30pm
To: "Todd Burruss" , "user@cassandra.apache.org"
Subject: RE: node repair
one last comment about thesting this is i st
t again and try it again with more data.
thx
________
From: Stu Hood [stu.h...@rackspace.com]
Sent: Sunday, March 21, 2010 12:08 PM
To: user@cassandra.apache.org
Subject: RE: node repair
If you have debug logs from the run, would you mind opening a JIRA describing
the problem?
-Original Message-
It is much more likely that you always increase your cluster in size by a
certain large percentage. With a 10 node cluster, you are likely to add 5 nodes
at a time, and with a 100 node cluster you'll probably add 25 to 50 per batch.
-Original Message-
From: "Daniel Kluesing"
Sent: Thur
replication factor == 1 means that there is only one copy of the data. And you
deleted it. Repair depends on the replication factor being greater than 1.
-Original Message-
From: "Jianing Hu"
Sent: Friday, March 26, 2010 9:33pm
To: user@cassandra.apache.org
Subject: Re: Is ReplicationFac
Ack... very sorry. I read the original message too quickly.
The fact that neither read-repair nor anti-entropy are working is suspicious
though. Do you think you could paste your config somewhere?
-Original Message-
From: "Stu Hood"
Sent: Friday, March 26, 2010 11:57p
64
periodic
1
864000
256
On Fri, Mar 26, 2010 at 10:00 PM, Stu Hood wrote:
> Ack... very sorry. I read the original message too quickly.
>
> The fact that neither read-repair nor anti-entropy are working is suspicious
> though. Do you think you could paste your config s
Eventually the new file format will make it in with #674, and we'll be able to
implement an option to skip corrupted data:
https://issues.apache.org/jira/browse/CASSANDRA-808
We're not ignoring this issue.
-Original Message-
From: "David Timothy Strauss"
Sent: Wednesday, March 31, 2010
48)
at
org.apache.cassandra.io.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:115)
I also tried in version 0.6, but above error still exist.
Perhaps, I will tried the way David Timothy suggest.
@Stu Hood: Do you implement code for 808 issues?
Thank a lot for support.
On Wed, Mar 31, 2010 a
Please read the README in the contrib/word_count directory.
-Original Message-
From: "Sonny Heer"
Sent: Wednesday, April 7, 2010 6:33pm
To: user@cassandra.apache.org
Subject: Re: Iterate through entire data set
Jon,
I've got the word_count.jar and a Hadoop cluster. How do you usually
ru
Code that uses Hadoop will look for mapred-site.xml, core-site.xml,
hdfs-site.xml etc on your CLASSPATH. If you add your Hadoop config directory to
CLASSPATH before running the script, Hadoop will use that configuration to
connect to your cluster.
-Original Message-
From: "Sonny Heer"
http://twitter.com/jromeh/status/12295736793
-Original Message-
From: "Mike Gallamore"
Sent: Friday, April 16, 2010 3:46pm
To: user@cassandra.apache.org
Subject: Re: Regarding Cassandra Scalability
Also people with 1M followers tend to have "public" tweets, which means
really I think it
elsewhere.
>>
>> Mason
>>
>
> http://wiki.apache.org/cassandra/CassandraLimitations
>
> We aren't hiding anything from the user who wishes to educate themselves.
>
> -Brandon
>
>
I didn't mean to imply anyone was hiding information. I was poin
ColumnFamilyInputFormat no longer uses the fat client API, and instead uses
Thrift. There are still some significant problems with the fat client, so it
shouldn't be used without a good understanding of those problems.
If you still want to use it, check out contrib/bmt_example, but I'd recommend
4:53pm
To: user@cassandra.apache.org
Subject: Re: Help with MapReduce
Where is the ColumnFamilyInputFormat that uses Thrift? I don't actually
have a preference about client, I just want to be consistent with
ColumnInputFormat.
On Sun, Apr 18, 2010 at 5:37 PM, Stu Hood wrote:
> ColumnFa
If you used that snippet of code, all connections would go through the same
seed: the input code does additional work to determine which nodes are holding
particular key ranges, and then connects directly.
For outputting from Hadoop to Cassandra, you may want to consider using a Java
clie
Nicolas,
Were all of those super column writes going to the same row?
http://wiki.apache.org/cassandra/CassandraLimitations
Thanks,
Stu
-Original Message-
From: "Nicolas Labrot"
Sent: Wednesday, April 21, 2010 11:54am
To: user@cassandra.apache.org
Subject: Re: Cassandra tuning for runn
It isn't very well documented apparently, but if you are using 0.6, you can
look at the 'Authenticator' property in the default config for an explanation
of how to authenticate users.
With the SimpleAuthenticator implementation, there are properties files that
define your users and passwords, a
Your keys cannot be an encoded as binary for OPP, since Cassandra will attempt
to decode them as UTF-8, meaning that they may not come back in the same format.
0.7 supports byte keys using the ByteOrderedPartitioner, and tokens are
specified using hex.
-Original Message-
From: "Mark Jon
The indexes within rows are _not_ implemented with Lucene: there is a custom
index structure that allows for random access within a row. But, you should
probably read http://wiki.apache.org/cassandra/CassandraLimitations to
understand the current limitations of the file format, some of which are
94 matches
Mail list logo