Re: Is SuperColumn necessary?
On Thu, May 6, 2010 at 5:38 PM, Vijay vijay2...@gmail.com wrote: I would rather be interested in Tree type structure where supercolumns have supercolumns in it. you dont need to compare all the columns to find a set of columns and will also reduce the bytes transfered for separator, at least string concatenation (Or something like that) for read and write column name generation. it is more logically stored and structured by this way and also we can make caching work better by selectively caching the tree (User defined if you will) But nothing wrong in supporting both :) I'm 99% sure we're talking about the same thing and we don't need to support both. How names/values are separated is pretty irrelevant. It has to happen somewhere. I agree that it'd be nice if it happened on the server, but doing it in the client makes it easier to explore ideas. On Thu, May 6, 2010 at 5:27 PM, philip andrew philip14...@gmail.com wrote: Please create a new term word if the existing terms are misleading, if its not a file system then its not good to call it a file system. While it's seriously bikesheddy, I guess you're right. Let's call them thingies for now, then. So you can have a top-level thingy and it can have an arbitrarily nested tree of sub-thingies. Each thingy has a thingy type [1]. You can also tell Cassandra if you want a particular level of thingy to be indexed. At one (or maybe more) levels you can tell Cassandra you want your thingies to be split onto separate nodes in your cluster. At one (or maybe more) levels you could also tell Cassandra that you want your thingies split into separate files [2]. The upshot is, the Cassandra data model would go from being it's a nested dictionary, just kidding no it's not! to being it's a nested dictionary, for serious. Again, these are all just ideas... but I think this simplified data model would allow you to express pretty much any query in a graph of simple primitives like Predicates, Filters, Aggregations, Transformations, etc. The indexes would allow you to cheat when evaluating certain types of queries - if you get a SlicePredicate on an indexed thingy you don't have to enumerate the entire set of sub-thingies for example. So, you'd query your thingies by building out a predicate, transformations, filters, etc., serializing the graph of primitives, and sending it over the wire to Cassandra. Cassandra would rebuild the graph and run it over your dataset. So instead of: Cassandra.get_range_slices( keyspace=AwesomeApp, column_parent=ColumnParent(column_family=user), slice_predicate=SlicePredicate(column_names=['username', 'dob']), range=KeyRange(start_key='a', end_key='m'), consistency_level=ONE ) You'd do something like: Cassandra.query( SubThingyTransformer( NamePredicate(names=[AwesomeApp], SubThingyTransformer( NamePredicate(names=[user]), SubThingyTransformer( SlicePredicate(start=a, end=m), NamePredicate(names=[username, dob]) ) ) ), consistency_level=ONE ) Which seems complicated, but it's basically just [(user['username'], user['dob']) for user in Cassandra['AwesomeApp']['user'].slice('a', 'm')] and could probably be expressed that way in a client library. I think batch_mutate is awesome the way it is and should be the only way to insert/update data. I'd rename it mutate. So our interface becomes: Cassandra.query(query, consistency_level) Cassandra.mutate(mutation, consistency_level) Ta-da. Anyways, I was trying to avoid writing all of this out in prose and try mocking some of it up in code instead. I guess this this works too. Either way, I do think something like this would simplify the codebase, simplify the data model, simplify the interface, make the entire system more flexible, and be generally awesome. Mike [1] These can be subclasses of Thingy in Java... or maybe they'd implement IThingy. But either way they'd handle serialization and probably implement compareTo to define natural ordering. So you'd have classes like ASCIIThingy, UTF8Thingy, and LongThingy (ahem) - these would replace comparators. [2] I think there's another simplification here. Splitting into separate files is really very similar to splitting onto separate nodes. There might be a way around some of the row size limitations with this sort of concept. And we may be able to get better utilization of multiple disks by giving each disk (or data directory) a subset of the node's token range. Caveat: thought not fully baked.
timeout while running simple hadoop job
Hi everyone, I am trying to develop a mapreduce job that does a simple selection+filter on the rows in our store. Of course it is mostly based on the WordCount example :) Sadly, while it seems the app runs fine on a test keyspace with little data, when run on a larger test index (but still on a single node) I reliably see this error in the logs 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001 java.lang.RuntimeException: TimedOutException() at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91) at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423) at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176) Caused by: TimedOutException() at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015) at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623) at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142) ... 11 more and after that the job seems to finish normally but no results are produced. FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if it ain't broke don't fix it). The single node has a data directory of about 127GB in two column families, off which the one used in the mapred job is about 100GB. The cassandra server is run with 6GB of heap on a box with 8GB available and no swap enabled. read/write latency from cfstat are Read Latency: 0.8535837762577986 ms. Write Latency: 0.028849603764075547 ms. row cache is not enabled, key cache percentage is default. Load on the machine is basically zero when the job is not running. As my code is 99% that from the wordcount contrib, I shall notice that In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we can supposedly change, but it's apparently not used anywhere, but as I said, running on a single node this should not be an issue anyway. Does anyone has suggestions or has seen this error before? On the other hand, did people run this kind of jobs in similar conditions flawlessly, so I can consider it just my problem? Thanks in advance for any help.
Re: Cassandra training on May 21 in Palo Alto
toronto :) If not toronto, Virginia. On Thu, May 6, 2010 at 5:28 PM, Jonathan Ellis jbel...@gmail.com wrote: We're planning that now. Where would you like to see one? On Thu, May 6, 2010 at 2:40 PM, S Ahmed sahmed1...@gmail.com wrote: Do you have rough ideas when you would be doing the next one? Maybe in 1 or 2 months or much later? On Tue, May 4, 2010 at 8:50 PM, Jonathan Ellis jbel...@gmail.com wrote: Yes, although when and where are TBD. On Tue, May 4, 2010 at 7:38 PM, Mark Greene green...@gmail.com wrote: Jonathan, Awesome! Any plans to offer this training again in the future for those of us who can't make it this time around? -Mark On Tue, May 4, 2010 at 5:07 PM, Jonathan Ellis jbel...@gmail.com wrote: I'll be running a day-long Cassandra training class on Friday, May 21. I'll cover - Installation and configuration - Application design - Basics of Cassandra internals - Operations - Tuning and troubleshooting Details at http://riptanobayarea20100521.eventbrite.com/ -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com
Re: bloom filter
On 2010-05-07 10:51, vineet daniel wrote: what is the benefit of creating bloom filter when cassandra writes data, how does it helps ? http://wiki.apache.org/cassandra/ArchitectureOverview -- David Strauss | da...@fourkitchens.com Four Kitchens | http://fourkitchens.com | +1 512 454 6659 [office] | +1 512 870 8453 [direct] signature.asc Description: OpenPGP digital signature
Re: Cassandra training on May 21 in Palo Alto
Reston, VA is a good spot in the DC metro area for tech events. The recent Pragmatic Programmer Clojure class sold out and already has two more return visits planned. On May 7, 2010, at 6:42 AM, S Ahmed sahmed1...@gmail.com wrote: toronto :) If not toronto, Virginia. On Thu, May 6, 2010 at 5:28 PM, Jonathan Ellis jbel...@gmail.com wrote: We're planning that now. Where would you like to see one? On Thu, May 6, 2010 at 2:40 PM, S Ahmed sahmed1...@gmail.com wrote: Do you have rough ideas when you would be doing the next one? Maybe in 1 or 2 months or much later? On Tue, May 4, 2010 at 8:50 PM, Jonathan Ellis jbel...@gmail.com wrote: Yes, although when and where are TBD. On Tue, May 4, 2010 at 7:38 PM, Mark Greene green...@gmail.com wrote: Jonathan, Awesome! Any plans to offer this training again in the future for those of us who can't make it this time around? -Mark On Tue, May 4, 2010 at 5:07 PM, Jonathan Ellis jbel...@gmail.com wrote: I'll be running a day-long Cassandra training class on Friday, May 21. I'll cover - Installation and configuration - Application design - Basics of Cassandra internals - Operations - Tuning and troubleshooting Details at http://riptanobayarea20100521.eventbrite.com/ -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com
Re: bloom filter
what is the benefit of creating bloom filter when cassandra writes data, how does it helps ? It allows Cassandra to answer requests for non-existent keys without going to disk, except in cases where the bloom filter gives a false positive. See: http://spyced.blogspot.com/2009/01/all-you-ever-wanted-to-know-about.html -- / Peter Schuller aka scode
Re: bloom filter
Thanks David and Peter. Is there any way to view the content of this file. ___ Vineet Daniel ___ Let your email find you On Fri, May 7, 2010 at 4:24 PM, David Strauss da...@fourkitchens.comwrote: On 2010-05-07 10:51, vineet daniel wrote: what is the benefit of creating bloom filter when cassandra writes data, how does it helps ? http://wiki.apache.org/cassandra/ArchitectureOverview -- David Strauss | da...@fourkitchens.com Four Kitchens | http://fourkitchens.com | +1 512 454 6659 [office] | +1 512 870 8453 [direct]
Re: bloom filter
On 2010-05-07 10:55, Peter Schüller wrote: what is the benefit of creating bloom filter when cassandra writes data, how does it helps ? It allows Cassandra to answer requests for non-existent keys without going to disk, except in cases where the bloom filter gives a false positive. See: http://spyced.blogspot.com/2009/01/all-you-ever-wanted-to-know-about.html It is also important for identifying which SSTable files to look inside even when a key is present. -- David Strauss | da...@fourkitchens.com Four Kitchens | http://fourkitchens.com | +1 512 454 6659 [office] | +1 512 870 8453 [direct] signature.asc Description: OpenPGP digital signature
Re: bloom filter
1. Peter said 'without going to disk' so that means bloom filters reside in memory, always or just when request to that particular CF is made. 2. It is also important for identifying which SSTable files to look inside even when a key is present. - David can you please throw some more light on your point, like what are the implications, why do we need to identify etc. ___ Vineet Daniel ___ Let your email find you On Fri, May 7, 2010 at 4:28 PM, David Strauss da...@fourkitchens.comwrote: On 2010-05-07 10:55, Peter Schüller wrote: what is the benefit of creating bloom filter when cassandra writes data, how does it helps ? It allows Cassandra to answer requests for non-existent keys without going to disk, except in cases where the bloom filter gives a false positive. See: http://spyced.blogspot.com/2009/01/all-you-ever-wanted-to-know-about.html It is also important for identifying which SSTable files to look inside even when a key is present. -- David Strauss | da...@fourkitchens.com Four Kitchens | http://fourkitchens.com | +1 512 454 6659 [office] | +1 512 870 8453 [direct]
Re: bloom filter
On 2010-05-07 11:03, vineet daniel wrote: 2. It is also important for identifying which SSTable files to look inside even when a key is present. - David can you please throw some more light on your point, like what are the implications, why do we need to identify etc. A bloom filter is almost like a street sign that tells you the range of addresses on a street block. Such a street sign doesn't guarantee the whole range of addresses exists on the block, but it does mean you can avoid driving down streets that don't contain the address you're looking for. When Cassandra is looking for a key, there could be several files that potentially contain it. By looking at the bloom filter for each, it can avoid looking inside the files that definitely do not have the desired data. (My analogy breaks down a bit here because the street signs indicate mutually exclusive ranges of addresses, while the bloom filters may indicate the possible presence of a key in *several* files.) -- David Strauss | da...@fourkitchens.com Four Kitchens | http://fourkitchens.com | +1 512 454 6659 [office] | +1 512 870 8453 [direct] signature.asc Description: OpenPGP digital signature
Re: pagination through slices with deleted keys
+1. There is some disagreement on whether or not the API should return empty columns or skip rows when no data is found. In all of our use cases, we would prefer skipped rows. And based on how frequently new cassandra users appear to be confused about the current behaviour, this might be a more common use case than the need for empty cols. Perhaps this could be added as an option on SlicePredicate ? (e.g. skipEmpty=true). On Fri, May 7, 2010 at 12:59 AM, Mike Malone m...@simplegeo.com wrote: On Thu, May 6, 2010 at 3:27 PM, Ian Kallen spidaman.l...@gmail.com wrote: Cool, is this a patch you've applied on the server side? Are you running 0.6.x? I'm wondering if this kind of thing can make it into future versions of Cassandra. Yea, server side. It's basically doing the same thing clients typically want to do (again, at least for our use cases) but doing it closer to the data. Our patch is kind of janky though. I can probably get some version of it pushed back upstream - or at least on github or something - if there's any interest. Mike
Re: timeout while running simple hadoop job
Huh? Isn't that the whole point of using Map/Reduce? On Fri, May 7, 2010 at 8:44 AM, Jonathan Ellis jbel...@gmail.com wrote: Sounds like you need to configure Hadoop to not create a whole bunch of Map tasks at once On Fri, May 7, 2010 at 3:47 AM, gabriele renzi rff@gmail.com wrote: Hi everyone, I am trying to develop a mapreduce job that does a simple selection+filter on the rows in our store. Of course it is mostly based on the WordCount example :) Sadly, while it seems the app runs fine on a test keyspace with little data, when run on a larger test index (but still on a single node) I reliably see this error in the logs 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001 java.lang.RuntimeException: TimedOutException() at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91) at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423) at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176) Caused by: TimedOutException() at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015) at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623) at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142) ... 11 more and after that the job seems to finish normally but no results are produced. FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if it ain't broke don't fix it). The single node has a data directory of about 127GB in two column families, off which the one used in the mapred job is about 100GB. The cassandra server is run with 6GB of heap on a box with 8GB available and no swap enabled. read/write latency from cfstat are Read Latency: 0.8535837762577986 ms. Write Latency: 0.028849603764075547 ms. row cache is not enabled, key cache percentage is default. Load on the machine is basically zero when the job is not running. As my code is 99% that from the wordcount contrib, I shall notice that In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we can supposedly change, but it's apparently not used anywhere, but as I said, running on a single node this should not be an issue anyway. Does anyone has suggestions or has seen this error before? On the other hand, did people run this kind of jobs in similar conditions flawlessly, so I can consider it just my problem? Thanks in advance for any help. -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com
Re: timeout while running simple hadoop job
There's also the mapred.task.timeout property that can be tweaked. But reporting is the correct way to fix timeouts during execution. On May 7, 2010, at 8:49 AM, Joseph Stein wrote: The problem could be that you are crunching more data than will be completed within the interval expire setting. In Hadoop you need to kind of tell the task tracker that you are still doing stuff which is done by setting status or incrementing counter on the Reporter object. http://allthingshadoop.com/2010/04/28/map-reduce-tips-tricks-your-first-real-cluster/ In your Java code there is a little trick to help the job be “aware” within the cluster of tasks that are not dead but just working hard. During execution of a task there is no built in reporting that the job is running as expected if it is not writing out. So this means that if your tasks are taking up a lot of time doing work it is possible the cluster will see that task as failed (based on the mapred.task.tracker.expiry.interval setting). Have no fear there is a way to tell cluster that your task is doing just fine. You have 2 ways todo this you can either report the status or increment a counter. Both of these will cause the task tracker to properly know the task is ok and this will get seen by the jobtracker in turn. Both of these options are explained in the JavaDoc http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/Reporter.html; Hope this helps On Fri, May 7, 2010 at 4:47 AM, gabriele renzi rff@gmail.com wrote: Hi everyone, I am trying to develop a mapreduce job that does a simple selection+filter on the rows in our store. Of course it is mostly based on the WordCount example :) Sadly, while it seems the app runs fine on a test keyspace with little data, when run on a larger test index (but still on a single node) I reliably see this error in the logs 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001 java.lang.RuntimeException: TimedOutException() at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91) at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423) at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176) Caused by: TimedOutException() at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015) at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623) at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142) ... 11 more and after that the job seems to finish normally but no results are produced. FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if it ain't broke don't fix it). The single node has a data directory of about 127GB in two column families, off which the one used in the mapred job is about 100GB. The cassandra server is run with 6GB of heap on a box with 8GB available and no swap enabled. read/write latency from cfstat are Read Latency: 0.8535837762577986 ms. Write Latency: 0.028849603764075547 ms. row cache is not enabled, key cache percentage is default. Load on the machine is basically zero when the job is not running. As my code is 99% that from the wordcount contrib, I shall notice that In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we can supposedly change, but it's apparently not used anywhere, but as I said, running on a single node this should not be an issue anyway. Does anyone has suggestions or has seen this error before? On the other hand, did people run this kind of jobs in similar conditions flawlessly, so I can consider it just my problem? Thanks in advance for any help. -- /* Joe Stein http://www.linkedin.com/in/charmalloc */
Re: timeout while running simple hadoop job
Joseph, the stacktrace suggests that it's Thrift that's timing out, not the Task. Gabriele, I believe that your problem is caused by too much load on Cassandra. Get_range_slices is presently an expensive operation. I had some success in reducing (although, it turns out, not eliminating) this problem by requesting smaller batches from get_range_slices. See ConfigHelper.setRangeBatchSize() joost On Fri, May 7, 2010 at 8:49 AM, Joseph Stein crypt...@gmail.com wrote: The problem could be that you are crunching more data than will be completed within the interval expire setting. In Hadoop you need to kind of tell the task tracker that you are still doing stuff which is done by setting status or incrementing counter on the Reporter object. http://allthingshadoop.com/2010/04/28/map-reduce-tips-tricks-your-first-real-cluster/ In your Java code there is a little trick to help the job be “aware” within the cluster of tasks that are not dead but just working hard. During execution of a task there is no built in reporting that the job is running as expected if it is not writing out. So this means that if your tasks are taking up a lot of time doing work it is possible the cluster will see that task as failed (based on the mapred.task.tracker.expiry.interval setting). Have no fear there is a way to tell cluster that your task is doing just fine. You have 2 ways todo this you can either report the status or increment a counter. Both of these will cause the task tracker to properly know the task is ok and this will get seen by the jobtracker in turn. Both of these options are explained in the JavaDoc http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/Reporter.html; Hope this helps On Fri, May 7, 2010 at 4:47 AM, gabriele renzi rff@gmail.com wrote: Hi everyone, I am trying to develop a mapreduce job that does a simple selection+filter on the rows in our store. Of course it is mostly based on the WordCount example :) Sadly, while it seems the app runs fine on a test keyspace with little data, when run on a larger test index (but still on a single node) I reliably see this error in the logs 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001 java.lang.RuntimeException: TimedOutException() at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91) at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423) at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176) Caused by: TimedOutException() at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015) at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623) at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142) ... 11 more and after that the job seems to finish normally but no results are produced. FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if it ain't broke don't fix it). The single node has a data directory of about 127GB in two column families, off which the one used in the mapred job is about 100GB. The cassandra server is run with 6GB of heap on a box with 8GB available and no swap enabled. read/write latency from cfstat are Read Latency: 0.8535837762577986 ms. Write Latency: 0.028849603764075547 ms. row cache is not enabled, key cache percentage is default. Load on the machine is basically zero when the job is not running. As my code is 99% that from the wordcount contrib, I shall notice that In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we can supposedly change, but it's apparently not used anywhere, but as I said, running on a single node this should not be an issue anyway. Does anyone has suggestions or has seen this error before? On the other hand, did people run this kind of jobs in
Re: timeout while running simple hadoop job
The whole point is to parallelize to use the available capacity across multiple machines. If you go past that point (fairly easy when you have a single machine) then you're just contending for resources, not making things faster. On Fri, May 7, 2010 at 7:48 AM, Joost Ouwerkerk jo...@openplaces.org wrote: Huh? Isn't that the whole point of using Map/Reduce? On Fri, May 7, 2010 at 8:44 AM, Jonathan Ellis jbel...@gmail.com wrote: Sounds like you need to configure Hadoop to not create a whole bunch of Map tasks at once On Fri, May 7, 2010 at 3:47 AM, gabriele renzi rff@gmail.com wrote: Hi everyone, I am trying to develop a mapreduce job that does a simple selection+filter on the rows in our store. Of course it is mostly based on the WordCount example :) Sadly, while it seems the app runs fine on a test keyspace with little data, when run on a larger test index (but still on a single node) I reliably see this error in the logs 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001 java.lang.RuntimeException: TimedOutException() at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91) at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423) at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176) Caused by: TimedOutException() at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015) at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623) at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597) at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142) ... 11 more and after that the job seems to finish normally but no results are produced. FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if it ain't broke don't fix it). The single node has a data directory of about 127GB in two column families, off which the one used in the mapred job is about 100GB. The cassandra server is run with 6GB of heap on a box with 8GB available and no swap enabled. read/write latency from cfstat are Read Latency: 0.8535837762577986 ms. Write Latency: 0.028849603764075547 ms. row cache is not enabled, key cache percentage is default. Load on the machine is basically zero when the job is not running. As my code is 99% that from the wordcount contrib, I shall notice that In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we can supposedly change, but it's apparently not used anywhere, but as I said, running on a single node this should not be an issue anyway. Does anyone has suggestions or has seen this error before? On the other hand, did people run this kind of jobs in similar conditions flawlessly, so I can consider it just my problem? Thanks in advance for any help. -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com
Re: timeout while running simple hadoop job
On Fri, May 7, 2010 at 3:02 PM, Joost Ouwerkerk jo...@openplaces.org wrote: Joseph, the stacktrace suggests that it's Thrift that's timing out, not the Task. Gabriele, I believe that your problem is caused by too much load on Cassandra. Get_range_slices is presently an expensive operation. I had some success in reducing (although, it turns out, not eliminating) this problem by requesting smaller batches from get_range_slices. See ConfigHelper.setRangeBatchSize() interesting, I will try and report.
Re: timeout while running simple hadoop job
On Fri, May 7, 2010 at 2:53 PM, Matt Revelle mreve...@gmail.com wrote: There's also the mapred.task.timeout property that can be tweaked. But reporting is the correct way to fix timeouts during execution. re: not reporting, I thought this was not needed with the new mapred api (Mapper class vs Mapper interface), plus I can see that the mappers do work, report percentage and happily terminate (I believe Mapper.Context.write does a call to progress() behind the scenes). I will check if it makes any difference though.
Re: timeout while running simple hadoop job
On Fri, May 7, 2010 at 2:44 PM, Jonathan Ellis jbel...@gmail.com wrote: Sounds like you need to configure Hadoop to not create a whole bunch of Map tasks at once interesting, from a quick check it seems there are a dozen threads running. Yet , setNumMapTasks seems to be deprecated (together with JobConf) and while I guess -Dmapred.map.tasks=N may still work, it seems that so it seems the only way to manage the number of map tasks is via a custom subclass of ColumnFamilyInputFormat. But of course you have a point that in a single box this does not add anything.
Re: Updating (as opposed to just setting) Cassandra data via Hadoop
On 5/6/10 3:26 PM, Stu Hood wrote: 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. Interesting, I'm trying to understand the performance impact of the different ways to do this. Under Mark's approach, the prior values are pulled out of Cassandra in the mapper in bulk, then merged and written back to Cassandra in the reducer; the get_range_slice is faster than the individual row fetches that my approach does in the reducer. Is that what you mean or are you referring to something else? thanks! -Ian -- Ian Kallen blog: http://www.arachna.com/roller/spidaman tweetz: http://twitter.com/spidaman vox: 925.385.8426
Re: timeout while running simple hadoop job
you can manage the number of map tasks by node mapred.tasktracker.map.tasks.maximum=1 On Fri, May 7, 2010 at 9:53 AM, gabriele renzi rff@gmail.com wrote: On Fri, May 7, 2010 at 2:44 PM, Jonathan Ellis jbel...@gmail.com wrote: Sounds like you need to configure Hadoop to not create a whole bunch of Map tasks at once interesting, from a quick check it seems there are a dozen threads running. Yet , setNumMapTasks seems to be deprecated (together with JobConf) and while I guess -Dmapred.map.tasks=N may still work, it seems that so it seems the only way to manage the number of map tasks is via a custom subclass of ColumnFamilyInputFormat. But of course you have a point that in a single box this does not add anything. -- /* Joe Stein http://www.linkedin.com/in/charmalloc */
Re: Cassandra training on May 21 in Palo Alto
It would be great if you could make a video of this event. Yes it won't like being there 1-1, but it sure would help get up to speed. On Fri, May 7, 2010 at 6:56 AM, Matt Revelle mreve...@gmail.com wrote: Reston, VA is a good spot in the DC metro area for tech events. The recent Pragmatic Programmer Clojure class sold out and already has two more return visits planned. On May 7, 2010, at 6:42 AM, S Ahmed sahmed1...@gmail.comsahmed1...@gmail.com sahmed1...@gmail.com wrote: toronto :) If not toronto, Virginia. On Thu, May 6, 2010 at 5:28 PM, Jonathan Ellis jbel...@gmail.comjbel...@gmail.comjbel...@gmail.com jbel...@gmail.com wrote: We're planning that now. Where would you like to see one? On Thu, May 6, 2010 at 2:40 PM, S Ahmed sahmed1...@gmail.comsahmed1...@gmail.comsahmed1...@gmail.com sahmed1...@gmail.com wrote: Do you have rough ideas when you would be doing the next one? Maybe in 1 or 2 months or much later? On Tue, May 4, 2010 at 8:50 PM, Jonathan Ellis jbel...@gmail.comjbel...@gmail.comjbel...@gmail.com jbel...@gmail.com wrote: Yes, although when and where are TBD. On Tue, May 4, 2010 at 7:38 PM, Mark Greene green...@gmail.comgreen...@gmail.comgreen...@gmail.com green...@gmail.com wrote: Jonathan, Awesome! Any plans to offer this training again in the future for those of us who can't make it this time around? -Mark On Tue, May 4, 2010 at 5:07 PM, Jonathan Ellis jbel...@gmail.comjbel...@gmail.comjbel...@gmail.com jbel...@gmail.com wrote: I'll be running a day-long Cassandra training class on Friday, May 21. I'll cover - Installation and configuration - Application design - Basics of Cassandra internals - Operations - Tuning and troubleshooting Details at http://riptanobayarea20100521.eventbrite.com/http://riptanobayarea20100521.eventbrite.com/http://riptanobayarea20100521.eventbrite.com/ http://riptanobayarea20100521.eventbrite.com/ -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com http://riptano.com http://riptano.com http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com http://riptano.com http://riptano.com http://riptano.com -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com http://riptano.com http://riptano.com http://riptano.com
Re: Is SuperColumn necessary?
On Wed, 2010-05-05 at 11:31 -0700, Ed Anuff wrote: Follow-up from last weeks discussion, I've been playing around with a simple column comparator for composite column names that I put up on github. I'd be interested to hear what people think of this approach. http://github.com/edanuff/CassandraCompositeType Clever. I wonder what a useful abstraction in Hector or one of the other idiomatic clients would look like. -- Eric Evans eev...@rackspace.com
Re: pagination through slices with deleted keys
On Fri, May 7, 2010 at 5:29 AM, Joost Ouwerkerk jo...@openplaces.orgwrote: +1. There is some disagreement on whether or not the API should return empty columns or skip rows when no data is found. In all of our use cases, we would prefer skipped rows. And based on how frequently new cassandra users appear to be confused about the current behaviour, this might be a more common use case than the need for empty cols. Perhaps this could be added as an option on SlicePredicate ? (e.g. skipEmpty=true). That's exactly how we implemented it: struct SlicePredicate { 1: optional listbinary column_names, 2: optional SliceRange slice_range, 3: optional bool ignore_empty_rows=0, } Mike
Overfull node
I've got two (out of five) nodes on my cassandra ring that somehow got too full (e.g. over 60% disk space utilization). I've now gotten a few new machines added to the ring, but evertime one of the overfull nodes attempts to stream its data it runs out of diskspace... I've tried half a dozen different bad ideas of how to get things moving along a bit smoother, but am at a total loss at this point. Is there any good tricks to get cassandra to not need 2x the disk space to stream out, or is something else potentially going on that's causing me problems? Thanks,
Re: key is sorted?
Columns are sorted (see CompareWith/CompareSubcolumnsWith) keys are not. On 7 maj 2010, at 22.10em, AJ Chen wrote: I have a super column family for topic, key being the name of the topic. ColumnFamily Name=Topic CompareWith=UTF8Type ColumnType=Super CompareSubcolumnsWith=BytesType / When I retrieve the rows, the rows are not sorted by the key. Is the row key sorted in cassandra by default? -aj -- AJ Chen, PhD Chair, Semantic Web SIG, sdforum.org http://web2express.org twitter @web2express Palo Alto, CA, USA
Re: Overfull node
If you're using RackUnawareStrategy (the default replication strategy) then you can bootstrap manually fairly easily -- copy all the data (not system) sstables from an overfull machine to a new machine, assign the new one a token that gives it about half of the old node's range, then start it with autobootstrap OFF. Then run cleanup on both new and old nodes to remove the part of the data that belongs to the other. The downside vs real bootstrap is you can't do this safely while writes are coming in to the original node. You can reduce your read-only period by doing an intial scp, then doing a flush + rsync when you're ready to take it read only. (https://issues.apache.org/jira/browse/CASSANDRA-579 will make this problem obsolete for 0.7 but that doesn't help you on 0.6, of course.) On Fri, May 7, 2010 at 2:08 PM, David Koblas kob...@extra.com wrote: I've got two (out of five) nodes on my cassandra ring that somehow got too full (e.g. over 60% disk space utilization). I've now gotten a few new machines added to the ring, but evertime one of the overfull nodes attempts to stream its data it runs out of diskspace... I've tried half a dozen different bad ideas of how to get things moving along a bit smoother, but am at a total loss at this point. Is there any good tricks to get cassandra to not need 2x the disk space to stream out, or is something else potentially going on that's causing me problems? Thanks, -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of Riptano, the source for professional Cassandra support http://riptano.com
Re: key is sorted?
thanks, that works. -aj On Fri, May 7, 2010 at 1:17 PM, Stu Hood stu.h...@rackspace.com wrote: 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 ajc...@web2express.org Sent: Friday, May 7, 2010 3:10pm To: user@cassandra.apache.org Subject: key is sorted? I have a super column family for topic, key being the name of the topic. ColumnFamily Name=Topic CompareWith=UTF8Type ColumnType=Super CompareSubcolumnsWith=BytesType / When I retrieve the rows, the rows are not sorted by the key. Is the row key sorted in cassandra by default? -aj -- AJ Chen, PhD Chair, Semantic Web SIG, sdforum.org http://web2express.org twitter @web2express Palo Alto, CA, USA -- AJ Chen, PhD Chair, Semantic Web SIG, sdforum.org http://web2express.org twitter @web2express Palo Alto, CA, USA
Data Modeling Conundrum
List, I have a case where visitors to a site are tracked via a persistent cookie containing a guid. This cookie is created and set when missing. Some of these visitors are logged in, meaning a userId may also be available. What I’m looking to do is have a way to associate each userId with all of the guids that it has been seen with. Conceptually, this would identify the unique (device, browser) pairs for each userId. The catch is that I want to be able to retrieve the most-recently-seen N guids for a userId. One possible solution to this problem in SQL looks like this (made up on the fly): # Table schema CREATE TABLE UserGuid ( userId INT, guid VARCHAR, when TIMESTAMP, PRIMARY KEY( userId, guid ), INDEX( userId, when ) ); # For each request with guid G and userId U at time T INSERT INTO UserGuid ( userId, guid, when ) VALUES ( U, G, T ) ON DUPLICATE KEY UPDATE SET when = T; # To get most recent N guids for userId U SELECT guid FROM UserGuid WHERE userId = U ORDER BY when DESC LIMIT N; Hopefully I’ve sufficiently explained what I’m trying to do. Now on to solving this problem in Cassandra. I’ve been trying to find a way that allows both of the above operations to be performed efficiently. Updates are a breeze with a structure like this: // Row key is userId 12345 : { // Column name is guid ‘256fb890-5a4b-11df-a08a-0800200c9a66’ : { // Column timestamp is last time guid was seen timestamp : 387587235233 } } but getting the last N recently seen guids requires pulling all columns and sorting by timestamp. Retrievals can be done efficiently with a structure taking advantage of column sorting: // Row key is userId 12345 : { // Column name is last time guid was seen 387587235233 : { // Column value is guid value: ‘256fb890-5a4b-11df-a08a-0800200c9a66’ } } where we use a slice get on the row with limit N (and reverse order). However, updates involve pulling all columns to de-duplicate guid values. Neither solution is ideal, and so I present this to you fine gentlemen who have more experience modeling data in Cassandra than I. I would much prefer to avoid any solutions that require pulling an indeterminate amount of data for either operation. For the time being I am using the first method and only pulling the first M columns, sorting, and taking the top N (M = N). One thing I was thinking would be nice (if possible), is to have a column family where columns are either sorted by their timestamp, or by the time the column was created/updated (which may be equivalent to not sorting at all, but I have not looked at the implementation). I appreciate any feedback or suggestions you might have. - William
RE: Cassandra vs. Voldemort benchmark
i did a lot of comparisons between voldemort and cassandra and in the end i decided to go with cassandra. the main reason was recovery and balancing operations. on the surface voldemort is s*** hot fast, until you need to restore a node or add nodes. BDB (the default persistence solution) isn't very good at iterating over the entire key space. it creates a _lot_ of random seeking across the disk and is therefore very slow. for small data sets voldemort is a winner as you can have all your data cached. (of course i'm not even considering data modeling, just performance on get/put/delete) i do agree that voldemort and cassandra are first class projects and worthy of many production deployments. right tool for the job. From: Kristian Eide [kre...@gmail.com] Sent: Friday, May 07, 2010 8:09 PM To: user@cassandra.apache.org Subject: Cassandra vs. Voldemort benchmark There is a benchmark comparing Cassandra to Voldemort performance here: http://blog.medallia.com/2010/05/choosing_a_keyvalue_storage_sy.html -- Kristian
Re: BinaryMemtable and collisions
Yes. When you flush from BMT, its like any other SSTable. Cassandra will merge them through compaction. That's good news, thanks for clarifying! A few more related questions: Are there any problems with issuing the flush command directly from code at the end up a bulk insert? The BMT example mentions running nodetool, but poking around the Cassandra source seems to indicate it should be doable programmatically. Also, in my BMT prototype I've noticed that the JVM won't exit after completion, so I have to hard kill it (ctrl-c). A thread dump shows that some of Cassandra's network threads are still open, keeping the JVM from exiting. Some digging revealed that Cassandra isn't designed with a clean shutdown in mind, so perhaps such behavior is expected. Still, it is a bit unsettling since the cluster nodes log an error after I kill the client node. Is calling StorageService.stopClient enough to ensure that any client-side buffers are flushed and writes are completed? Finally, the wiki page on BMT ( http://wiki.apache.org/cassandra/BinaryMemtable) suggests using StorageProxy, but the example in contrib does not. Under the hood, both StorageProxy and the contrib example call MessagingService.sendOneWay. The additional code in StorageProxy seems mostly related to the extra bookkeeping associated with hinted handoff and waiting on write acks. Perhaps that extra work may not be entirely necessary for a bulk load operation? That should be enough questions from me for a while. :) -Toby
Benefits of using framed transport over non-framed transport?
Hi everyone, Can anyone throw a light at the benefits of using framed transport over non-framed transport? We are trying to sum up some performance tuning approaches of cassandra in our project. Can framed transport be counted? Thanks 2010-05-08
Re: BinaryMemtable and collisions
Any reason why you aren't using Lucandra directly? On Fri, May 7, 2010 at 8:21 PM, Tobias Jungen tobias.jun...@gmail.comwrote: Greetings, Started getting my feet wet with Cassandra in earnest this week. I'm building a custom inverted index of sorts on top of Cassandra, in part inspired by the work of Jake Luciani in Lucandra. I've successfully loaded nearly a million documents over a 3-node cluster, and initial query tests look promising. The problem is that our target use case has hundreds of millions of documents (each document is very small however). Loading time will be an important factor. I've investigated using the BinaryMemtable interface (as found in contrib/bmt_example) to speed up bulk insertion. I have a prototype up that successfully inserts data using BMT, but there is a problem. If I perform multiple writes for the same row key column family, the row ends up containing only one of the writes. I'm guessing this is because with BMT I need to group all writes for a given row key column family into one operation, rather than doing it incrementally as is possible with the thrift interface. Hadoop obviously is the solution for doing such a grouping. Unfortunately, we can't perform such a process over our entire dataset, we will need to do it in increments. So my question is: If I properly flush every node after performing a larger bulk insert, can Cassandra merge multiple writes on a single row column family when using the BMT interface? Or is using BMT only feasible for loading data on rows that don't exist yet? Thanks in advance, Toby Jungen
Re: BinaryMemtable and collisions
Without going into too much depth: Our retrieval model is a bit more structured than standard lucene retrieval, and I'm trying to leverage that structure. Some of the terms we're going to retrieve against have high occurrence, and because of that I'm worried about getting killed by processing large term vectors. Instead I'm trying to index on term relationships, if that makes sense. On Sat, May 8, 2010 at 12:09 AM, Jake Luciani jak...@gmail.com wrote: Any reason why you aren't using Lucandra directly? On Fri, May 7, 2010 at 8:21 PM, Tobias Jungen tobias.jun...@gmail.comwrote: Greetings, Started getting my feet wet with Cassandra in earnest this week. I'm building a custom inverted index of sorts on top of Cassandra, in part inspired by the work of Jake Luciani in Lucandra. I've successfully loaded nearly a million documents over a 3-node cluster, and initial query tests look promising. The problem is that our target use case has hundreds of millions of documents (each document is very small however). Loading time will be an important factor. I've investigated using the BinaryMemtable interface (as found in contrib/bmt_example) to speed up bulk insertion. I have a prototype up that successfully inserts data using BMT, but there is a problem. If I perform multiple writes for the same row key column family, the row ends up containing only one of the writes. I'm guessing this is because with BMT I need to group all writes for a given row key column family into one operation, rather than doing it incrementally as is possible with the thrift interface. Hadoop obviously is the solution for doing such a grouping. Unfortunately, we can't perform such a process over our entire dataset, we will need to do it in increments. So my question is: If I properly flush every node after performing a larger bulk insert, can Cassandra merge multiple writes on a single row column family when using the BMT interface? Or is using BMT only feasible for loading data on rows that don't exist yet? Thanks in advance, Toby Jungen
Re: BinaryMemtable and collisions
Got it. I'm working on making term vectors optional and just store frequency in this case. Just FYI. On Sat, May 8, 2010 at 1:17 AM, Tobias Jungen tobias.jun...@gmail.comwrote: Without going into too much depth: Our retrieval model is a bit more structured than standard lucene retrieval, and I'm trying to leverage that structure. Some of the terms we're going to retrieve against have high occurrence, and because of that I'm worried about getting killed by processing large term vectors. Instead I'm trying to index on term relationships, if that makes sense. On Sat, May 8, 2010 at 12:09 AM, Jake Luciani jak...@gmail.com wrote: Any reason why you aren't using Lucandra directly? On Fri, May 7, 2010 at 8:21 PM, Tobias Jungen tobias.jun...@gmail.comwrote: Greetings, Started getting my feet wet with Cassandra in earnest this week. I'm building a custom inverted index of sorts on top of Cassandra, in part inspired by the work of Jake Luciani in Lucandra. I've successfully loaded nearly a million documents over a 3-node cluster, and initial query tests look promising. The problem is that our target use case has hundreds of millions of documents (each document is very small however). Loading time will be an important factor. I've investigated using the BinaryMemtable interface (as found in contrib/bmt_example) to speed up bulk insertion. I have a prototype up that successfully inserts data using BMT, but there is a problem. If I perform multiple writes for the same row key column family, the row ends up containing only one of the writes. I'm guessing this is because with BMT I need to group all writes for a given row key column family into one operation, rather than doing it incrementally as is possible with the thrift interface. Hadoop obviously is the solution for doing such a grouping. Unfortunately, we can't perform such a process over our entire dataset, we will need to do it in increments. So my question is: If I properly flush every node after performing a larger bulk insert, can Cassandra merge multiple writes on a single row column family when using the BMT interface? Or is using BMT only feasible for loading data on rows that don't exist yet? Thanks in advance, Toby Jungen