Re: get_range_slices confused about token ranges after decommissioning a node

2010-06-22 Thread Joost Ouwerkerk
/Streaming > > On Tue, Jun 22, 2010 at 12:57 AM, Joost Ouwerkerk > wrote: > > Yes, although "forget" implies that we once knew we were supposed to do > so. > > Given the following before-and-after states, on which nodes are we > supposed > > to run repair?

Re: get_range_slices confused about token ranges after decommissioning a node

2010-06-21 Thread Joost Ouwerkerk
run repair? > > On Mon, Jun 21, 2010 at 7:02 PM, Joost Ouwerkerk > wrote: > > I believe we did nodetool removetoken on nodes that were already down > (due > > to hardware failure), but I will check to make sure. We're running > Cassandra > > 0.6.2. > > &

Re: get_range_slices confused about token ranges after decommissioning a node

2010-06-21 Thread Joost Ouwerkerk
I should add that we have a replication factor of 3 and a cluster with 30 nodes. On Mon, Jun 21, 2010 at 10:02 PM, Joost Ouwerkerk wrote: > I believe we did nodetool removetoken on nodes that were already down (due > to hardware failure), but I will check to make sure. We're runnin

Re: get_range_slices confused about token ranges after decommissioning a node

2010-06-21 Thread Joost Ouwerkerk
I believe we did nodetool removetoken on nodes that were already down (due to hardware failure), but I will check to make sure. We're running Cassandra 0.6.2. On Mon, Jun 21, 2010 at 9:59 PM, Joost Ouwerkerk wrote: > Greg, can you describe the steps we took to decommission t

get_range_slices confused about token ranges after decommissioning a node

2010-06-21 Thread Joost Ouwerkerk
We're seeing very strange behaviour after decommissioning a node: when requesting a get_range_slices with a KeyRange by token, we are getting back tokens that are out of range. As a result, ColumnFamilyRecordReader gets confused, since it uses the last token from the result set to set the start tok

Re: Load Balancing Mapper Tasks

2010-05-17 Thread Joost Ouwerkerk
) java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:947) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907) java.lang.Thread.run(Thread.java:619) On Mon, May 17, 2010 at 12:53 AM, Jonathan Ellis wrote: > On Sun, May 16, 2010 at 2:52 PM, Joost Ouwerkerk > wrote: > &g

Re: Load Balancing Mapper Tasks

2010-05-16 Thread Joost Ouwerkerk
umed Hadoop would be smart enough to > load-balance the jobs it sends out. Guess not. > > Can you submit a patch? > > On Wed, May 12, 2010 at 12:32 PM, Joost Ouwerkerk > wrote: > > I've been trying to improve the time it takes to map 30 million rows > usi

Load Balancing Mapper Tasks

2010-05-12 Thread Joost Ouwerkerk
I've been trying to improve the time it takes to map 30 million rows using a hadoop / cassandra cluster with 30 nodes. I discovered that since CassandraInputFormat returns an ordered list of splits, when there are many splits (e.g. hundreds or more) the load on cassandra is horribly unbalanced. e

Re: timeout while running simple hadoop job

2010-05-07 Thread Joost Ouwerkerk
The number of map tasks for a job is a function of the InputFormat, which in the case of ColumnInputFormat is a function of the global number of keys in Cassandra. The number of concurrent maps being executed at any given time per TaskTracker (per node) is set by mapred.tasktracker.reduce.tasks.ma

Re: timeout while running simple hadoop job

2010-05-07 Thread Joost Ouwerkerk
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 prob

Re: timeout while running simple hadoop job

2010-05-07 Thread Joost Ouwerkerk
Huh? Isn't that the whole point of using Map/Reduce? On Fri, May 7, 2010 at 8:44 AM, Jonathan Ellis 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 wrote: >> Hi everyone, >> >> I am trying to

Re: pagination through slices with deleted keys

2010-05-07 Thread Joost Ouwerkerk
+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

Export to another cassandra cluster

2010-05-04 Thread Joost Ouwerkerk
I want to export data from one cassandra cluster (production) to another (development). This is not a case of replication, because I just want a snapshot, not a continuous synchronization. I guess my options include 'nodetool snapshot' and 'sstable2json'. In our case, however, the development cl

Re: Single Split ColumnFamilyRecordReader returns duplicate rows

2010-05-01 Thread Joost Ouwerkerk
Created CASSANDRA-1042. On Sat, May 1, 2010 at 12:01 AM, Jonathan Ellis wrote: > Can you create a ticket? > > On Fri, Apr 30, 2010 at 4:55 PM, Joost Ouwerkerk wrote: >> There's a bug in ColumnFamilyRecordReader that appears when processing >> a single split.  When t

Single Split ColumnFamilyRecordReader returns duplicate rows

2010-04-30 Thread Joost Ouwerkerk
There's a bug in ColumnFamilyRecordReader that appears when processing a single split. When the start and end tokens of the split are equal, duplicate rows can be returned. Example with 5 rows: token (start and end) = 53193025635115934196771903670925341736 Tokens returned by first get_range_slic

Re: Cassandra reverting deletes?

2010-04-30 Thread Joost Ouwerkerk
Great, thank you. Do you have a hypothesis about where things might be going wrong? Let me know what I can do to help. On Fri, Apr 30, 2010 at 9:33 AM, Jonathan Ellis wrote: > https://issues.apache.org/jira/browse/CASSANDRA-1040 > > On Thu, Apr 29, 2010 at 6:55 PM, Joost Ouwerke

Re: Cassandra reverting deletes?

2010-04-29 Thread Joost Ouwerkerk
an Ellis wrote: > Good! :) > > Can you reproduce w/o map/reduce, with raw get_range_slices? > > On Wed, Apr 28, 2010 at 3:56 PM, Joost Ouwerkerk wrote: >> Yes! Reproduced on single-node cluster: >> >> 10/04/28 16:30:24 INFO mapred.JobClient:     ROWS=274884 >> 1

Re: TimedOutException when using the ColumnFamilyInputFormat

2010-04-29 Thread Joost Ouwerkerk
The default batch size is 4096, which means that each call to get_range_slices retrieves 4,096 rows. I have found that this causes timeouts when cassandra is under load. Try reducing the batchsize with a call to ConfigHelper.setRangeBatchSize(). This has eliminated the TimedOutExceptions for us.

Re: Cassandra reverting deletes?

2010-04-28 Thread Joost Ouwerkerk
, 2010 at 10:43 AM, Jonathan Ellis wrote: > It sounds like either there is a fairly obvious bug, or you're doing > something wrong. :) > > Can you reproduce against a single node? > > On Tue, Apr 27, 2010 at 5:14 PM, Joost Ouwerkerk wrote: >> Update: I ran a test wher

Re: Cassandra reverting deletes?

2010-04-27 Thread Joost Ouwerkerk
52:34, finished 18:10:28 (17mins, 53 seconds) ROWS 1,425,644 TOMBSTONES 116,904 On Tue, Apr 27, 2010 at 5:37 PM, Joost Ouwerkerk wrote: > Clocks are in sync: > > cluster04:~/cassandra$ dsh -g development "date" > Tue Apr 27 17:36:33 EDT 2010 > Tue Apr 27 17:36:33

Re: Cassandra reverting deletes?

2010-04-27 Thread Joost Ouwerkerk
intentional read-repair occurring if > that were the case. > > -Nate > > On Tue, Apr 27, 2010 at 2:20 PM, Joost Ouwerkerk wrote: >> Hmm... Even after deleting with cl.ALL, I'm getting data back for some >> rows after having deleted them.  Which rows return data is >

Re: Cassandra reverting deletes?

2010-04-27 Thread Joost Ouwerkerk
Hmm... Even after deleting with cl.ALL, I'm getting data back for some rows after having deleted them. Which rows return data is inconsistent from one run of the job to the next. On Tue, Apr 27, 2010 at 1:44 PM, Joost Ouwerkerk wrote: > To check that rows are gone, I check that KeySlice

Re: Cassandra reverting deletes?

2010-04-27 Thread Joost Ouwerkerk
> Can you reproduce with normal QUORUM? > > On Sat, Apr 24, 2010 at 12:23 PM, Joost Ouwerkerk > wrote: > > I'm having trouble deleting rows in Cassandra. After running a job that > > deletes hundreds of rows, I run another job that verifies that the rows > are >

Re: New User: OSX vs. Debian on Cassandra 0.5.0 with Thrift

2010-04-24 Thread Joost Ouwerkerk
Socket classes > and you should see a performance improvement from buffering. > > > > On Apr 24, 2010, at 5:31 PM, Joost Ouwerkerk wrote: > > Is this something that also needs to be managed in Java? In most examples > I've seen, connections are created like this: &

Re: New User: OSX vs. Debian on Cassandra 0.5.0 with Thrift

2010-04-24 Thread Joost Ouwerkerk
Is this something that also needs to be managed in Java? In most examples I've seen, connections are created like this: TSocket socket = new TSocket(location, thriftport) TBinaryProtocol binaryProtocol = new TBinaryProtocol(socket, false, false); Cassandra.Clien

Cassandra reverting deletes?

2010-04-24 Thread Joost Ouwerkerk
I'm having trouble deleting rows in Cassandra. After running a job that deletes hundreds of rows, I run another job that verifies that the rows are gone. Both jobs run correctly. However, when I run the verification job an hour later, the rows have re-appeared. This is not a case of "ghosting"

Re: MapReduce, Timeouts and Range Batch Size

2010-04-23 Thread Joost Ouwerkerk
ot going to be an easy backport > since SSTableReader and SSTableScanner got split into two classes in > trunk. > > On Fri, Apr 23, 2010 at 9:39 AM, Joost Ouwerkerk > wrote: > > Awesome. In the meantime, I hacked something similar myself. The > > performance difference

Re: MapReduce, Timeouts and Range Batch Size

2010-04-23 Thread Joost Ouwerkerk
Ellis wrote: > > > >> That would be an easy win, sure. > >> > >> On Thu, Apr 22, 2010 at 9:27 PM, Joost Ouwerkerk > wrote: > >>> I was getting client timeouts in ColumnFamilyRecordReader.maybeInit() > when > >>> MapReducing. So I'

MapReduce, Timeouts and Range Batch Size

2010-04-22 Thread Joost Ouwerkerk
I was getting client timeouts in ColumnFamilyRecordReader.maybeInit() when MapReducing. So I've reduced the Range Batch Size to 256 (from 4096) and this seems to have fixed my problem, although it has slowed things down a bit -- presumably because there are 16x more calls to get_range_slices. Whil

Tool for managing cluster nodes?

2010-04-20 Thread Joost Ouwerkerk
What are people using to manage Cassandra cluster nodes? i.e. to start, stop, copy config files, etc. I'm using cssh and wondering if there is a better way... Joost.

Re: Help with MapReduce

2010-04-19 Thread Joost Ouwerkerk
Ok. This should be ok for now, although not optimal for some jobs. Next issue is node stability during the insert job. The stacktrace below occured on several nodes while inserting 10 million rows. We're running on 4G machines, 1G of which is allocated to cassandra. What's the best config to p

Re: Help with MapReduce

2010-04-19 Thread Joost Ouwerkerk
And when retrieving only one supercolumn? Can I further specify which subcolumns to retrieve? On Mon, Apr 19, 2010 at 9:29 PM, Jonathan Ellis wrote: > the latter, if you are retrieving multiple supercolumns. > > On Mon, Apr 19, 2010 at 8:10 PM, Joost Ouwerkerk > wrote: > >

Re: Help with MapReduce

2010-04-19 Thread Joost Ouwerkerk
bly there is something else causing poor performance, like swapping. > > On Mon, Apr 19, 2010 at 7:12 PM, Joost Ouwerkerk > wrote: > > I'm slowly getting somewhere with Cassandra... I have successfully > imported > > 1.5 million rows using MapReduce. This took about 8 m

Re: Help with MapReduce

2010-04-19 Thread Joost Ouwerkerk
; 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 that you use thrift for now. > > > > -Original Message- > > From: "

Re: Help with MapReduce

2010-04-18 Thread Joost Ouwerkerk
> 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 that you use thrift for now. > > --

Help with MapReduce

2010-04-18 Thread Joost Ouwerkerk
I'm a Cassandra noob trying to validate Cassandra as a viable alternative to HBase (which we've been using for over a year) for our application. So far, I've had no success getting Cassandra working with MapReduce. My first step is inserting data into Cassandra. I've created a MapRed job based u

Help with MapReduce

2010-04-18 Thread Joost Ouwerkerk
I'm a Cassandra noob trying to validate Cassandra as a viable alternative to HBase (which we've been using for over a year) for our application. So far, I've had no success getting Cassandra working with MapReduce. My first step is inserting data into Cassandra. I've created a MapRed job based u