about the compaction strategy upgrade

2012-05-09 Thread zhangcheng2
I have a cassandra cluster with 2T data at each server. I used 
SizeTieredCompactionStrategy, and I want to upgrade to LevelCompactionStrategy. 
How can I do? I try to update the strategy to LeveledCompactionStrategy and set 
the sstable size to 10mb, then all the files are in L0, this will cost a lot 
time to finish compaction. Are there any better ideas?

Thanks.




zhangcheng2

Re: sstableloader 1.1 won't stream

2012-05-09 Thread Sylvain Lebresne
Have you checked for errors in the servers' logs?

--
Sylvain

On Tue, May 8, 2012 at 1:24 PM, Pieter Callewaert
 wrote:
> I've updated all nodes to 1.1 but I keep getting the same problem...
> Any other thoughts about this?
>
> Kind regards,
> Pieter
>
> -Original Message-
> From: Benoit Perroud [mailto:ben...@noisette.ch]
> Sent: maandag 7 mei 2012 22:21
> To: user@cassandra.apache.org
> Subject: Re: sstableloader 1.1 won't stream
>
> You may want to upgrade all your nodes to 1.1.
>
> The streaming process connect to every living nodes of the cluster (you can 
> explicitely diable some nodes), so all nodes need to speak 1.1.
>
>
>
> 2012/5/7 Pieter Callewaert :
>> Hi,
>>
>>
>>
>> I’m trying to upgrade our bulk load process in our testing env.
>>
>> We use the SSTableSimpleUnsortedWriter to write tables, and use
>> sstableloader to stream it into our cluster.
>>
>> I’ve changed the writer program to fit to the 1.1 api, but now I’m
>> having troubles to load them to our cluster. The cluster exists out of
>> one 1.1 node and two 1.0.9 nodes.
>>
>>
>>
>> I’ve enabled debug as parameter and in the log4j conf.
>>
>>
>>
>> [root@bms-app1 ~]# ./apache-cassandra/bin/sstableloader --debug -d
>> 10.10.10.100 /tmp/201205071234/MapData024/HOS/
>>
>> INFO 16:25:40,735 Opening
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1 (1588949 bytes)
>>
>> INFO 16:25:40,755 JNA not found. Native methods will be disabled.
>>
>> DEBUG 16:25:41,060 INDEX LOAD TIME for
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1: 327 ms.
>>
>> Streaming revelant part of
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to
>> [/10.10.10.102, /10.10.10.100, /10.10.10.101]
>>
>> INFO 16:25:41,083 Stream context metadata
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,084 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,087 Streaming to /10.10.10.102
>>
>> DEBUG 16:25:41,092 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%
>>
>> INFO 16:25:41,099 Stream context metadata
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,100 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,100 Streaming to /10.10.10.100
>>
>> DEBUG 16:25:41,100 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%
>>
>> INFO 16:25:41,102 Stream context metadata
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,102 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,102 Streaming to /10.10.10.101
>>
>> DEBUG 16:25:41,102 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%
>>
>>
>>
>> progress: [/10.10.10.102 0/1 (0)] [/10.10.10.100 0/1 (0)]
>> [/10.10.10.101 0/1 (0)] [total: 0 - 0MB/s (avg: 0MB/s)] WARN
>> 16:25:41,107 Failed attempt 1 to connect to /10.10.10.101 to stream
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:41,108 Failed attempt 1 to connect to /10.10.10.102 to
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:41,108 Failed attempt 1 to connect to /10.10.10.100 to
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> progress: [/10.10.10.102 0/1 (0)] [/10.10.10.100 0/1 (0)]
>> [/10.10.10.101 0/1 (0)] [total: 0 - 0MB/s (avg: 0MB/s)] WARN
>> 16:25:45,109 Failed attempt 2 to connect to /10.10.10.101 to stream
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%. Retrying in 8000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:45,110 Failed attempt 2 to connect to /10.10.10.102 to
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%. Retrying in 8000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:45,110 Failed attempt 2 to connect to /10.10.10.100 to
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%.

Re: Timeout Exception in get_slice

2012-05-09 Thread aaron morton
How big are the multi get batches ?

How do the wide row get_slice calls behave when the multi gets are not running ?

Cheers

-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 9/05/2012, at 1:47 AM, Luís Ferreira wrote:

> Maybe one of the problems is that I am reading the columns in a row and the 
> rows themselves in batches, using the count attribute in the SliceRange and 
> by changing the start column or the corresponding for rows with the KeyRange. 
> According to your blog post, using start key to read for millions of 
> rows/columns has a lot of latency, but how else can I read an entire row that 
> does not fit into memory?
> 
> I'll have to run some tests again and check the tpstats. Still, do you think 
> that adding more machines to the cluster will help a lot? I say this, because 
> I started with a 3 node cluster and have scaled to a 5 node cluster with 
> little improvement... 
> 
> Thanks anyway.
> 
> On May 8, 2012, at 9:54 AM, aaron morton wrote:
> 
>> If I was rebuilding my power after spending the first thousand years of the 
>> Third Age as a shapeless evil I would cast my Eye of Fire in the direction 
>> of the filthy little multi_gets. 
>> 
>> A node can fail to respond to a query with rpc_timeout for two reasons: 
>> either the command did not run or the command started but did not complete. 
>> The former is much more likely. If it is happening you will see  large 
>> pending counts and dropped messages in nodetool tpstats, you will also see 
>> log entries about dropped messages.
>> 
>> When you send a multi_get each row you request becomes a message in the read 
>> thread pool. If you request 100 rows you will put 100 messages in the pool, 
>> which by default has 32 threads. If some clients are sending large multi get 
>> (or batch mutations) you can overload nodes and starve other clients. 
>> 
>> for background, some metrics here for selecting from 10million columns 
>> http://thelastpickle.com/2011/07/04/Cassandra-Query-Plans/
>> 
>> Hope that helps. 
>> 
>> 
>> -
>> Aaron Morton
>> Freelance Developer
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 6/05/2012, at 7:14 AM, Luís Ferreira wrote:
>> 
>>> Hi, 
>>> 
>>> I'm doing get_slice on huge rows (3 million columns) and even though I am 
>>> doing it iteratively I keep getting TimeoutExceptions. I've tried to change 
>>> the number of columns fetched but it did not work. 
>>> 
>>> I have a 5 machine cluster, each with 4GB of which 3 are dedicated to 
>>> cassandra's heap, but still the all consume all of the memory and get huge 
>>> IO wait due to the amout of reads.
>>> 
>>> I am running tests with 100 clients all performing multiple operations 
>>> mostly get_slice, get and multi_get, but the timeouts only occur in the 
>>> get_slice.
>>> 
>>> Does this have anything to do with cassandra's ability (or lack thereof) to 
>>> keep the rows in memory? Or am I doing anything wrong? Any tips?
>>> 
>>> Cumpliments,
>>> Luís Ferreira
>>> 
>>> 
>>> 
>>> 
>> 
> 
> Cumprimentos,
> Luís Ferreira
> 
> 
> 



Re: Error deleting column families with 1.1

2012-05-09 Thread André Cruz
Here it is: https://issues.apache.org/jira/browse/CASSANDRA-4230

Please let me know if you need further info.

Best regards,
André

On May 8, 2012, at 23:55 , aaron morton wrote:

> Could you please create a ticket for this on 
> https://issues.apache.org/jira/browse/CASSANDRA
> 
> Please include:
> * operating system
> * keyspace / column family definition
> * output of of "ls -lah" for the "/var/lib/cassandra/data/Disco/Client/" 
> directory after the error occurs.
> 
> Thanks
> 
> -
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 8/05/2012, at 9:00 AM, André Cruz wrote:
> 
>> Hello.
>> 
>> Since I upgraded to Cassandra 1.1, I get the following error when trying to 
>> delete a CF. After this happens the CF is not accessible anymore, but I 
>> cannot create another one with the same name until I restart the server.
>> 
>> INFO [MigrationStage:1] 2012-05-07 18:10:12,682 ColumnFamilyStore.java (line 
>> 634) Enqueuing flush of Memtable-schema_columnfamilies@1128094887(978/1222 
>> serialized/live bytes, 21 ops)
>> INFO [FlushWriter:2] 2012-05-07 18:10:12,682 Memtable.java (line 266) 
>> Writing Memtable-schema_columnfamilies@1128094887(978/1222 serialized/live 
>> bytes, 21 ops)
>> INFO [FlushWriter:2] 2012-05-07 18:10:12,720 Memtable.java (line 307) 
>> Completed flushing 
>> /var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-28-Data.db
>>  (1041 bytes)
>> INFO [MigrationStage:1] 2012-05-07 18:10:12,721 ColumnFamilyStore.java (line 
>> 634) Enqueuing flush of Memtable-schema_columns@1599271050(392/490 
>> serialized/live bytes, 8 ops)
>> INFO [FlushWriter:2] 2012-05-07 18:10:12,722 Memtable.java (line 266) 
>> Writing Memtable-schema_columns@1599271050(392/490 serialized/live bytes, 8 
>> ops)
>> INFO [CompactionExecutor:8] 2012-05-07 18:10:12,722 CompactionTask.java 
>> (line 114) Compacting 
>> [SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-26-Data.db'),
>> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-28-Data.db'),
>>  
>> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfam
>> ilies-hc-27-Data.db'), 
>> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-25-Data.db')]
>> INFO [FlushWriter:2] 2012-05-07 18:10:12,806 Memtable.java (line 307) 
>> Completed flushing 
>> /var/lib/cassandra/data/system/schema_columns/system-schema_columns-hc-23-Data.db
>>  (447 bytes)
>> INFO [CompactionExecutor:8] 2012-05-07 18:10:12,811 CompactionTask.java 
>> (line 225) Compacted to 
>> [/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-29-Data.db,].
>>   24,797 to 21,431
>> (~86% of original) bytes for 2 keys at 0.232252MB/s.  Time: 88ms.
>> ERROR [MigrationStage:1] 2012-05-07 18:10:12,895 CLibrary.java (line 158) 
>> Unable to create hard link
>> com.sun.jna.LastErrorException: errno was 17
>> at org.apache.cassandra.utils.CLibrary.link(Native Method)
>> at org.apache.cassandra.utils.CLibrary.createHardLink(CLibrary.java:150)
>> at 
>> org.apache.cassandra.db.Directories.snapshotLeveledManifest(Directories.java:343)
>> at 
>> org.apache.cassandra.db.ColumnFamilyStore.snapshotWithoutFlush(ColumnFamilyStore.java:1450)
>> at 
>> org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:1483)
>> at org.apache.cassandra.db.DefsTable.dropColumnFamily(DefsTable.java:512)
>> at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:403)
>> at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:270)
>> at 
>> org.apache.cassandra.service.MigrationManager$1.call(MigrationManager.java:214)
>> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>> at 
>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>> at 
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>> at java.lang.Thread.run(Thread.java:662)
>> ERROR [Thrift:17] 2012-05-07 18:10:12,898 CustomTThreadPoolServer.java (line 
>> 204) Error occurred during processing of message.
>> java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
>> java.io.IOError: java.io.IOException: Unable to create hard link from 
>> /var/lib/cassandra/data/Disco/Client/Client.json to /var/lib/cassandra/data/
>> Disco/Client/snapshots/1336410612893-Client/Client.json (errno 17)
>> at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:372)
>> at 
>> org.apache.cassandra.service.MigrationManager.announce(MigrationManager.java:191)
>> at 
>> org.apache.cassandra.service.MigrationManager.announceColumnFamilyDrop(MigrationManager.java:182)
>> at 
>> org.apache.cassandra.thrift.CassandraServer.system_drop_column_family(CassandraServer.java:948)
>> at 

Re: getting status of long running repair

2012-05-09 Thread Bill Au
I am running 1.0.8.  Two data center with 8 machines in each dc.  Nodes are
all up while repairing is running.  No dropped Mutations/Messages.  I do
see HintedHandoff messages.

Bill

On Tue, May 8, 2012 at 11:15 PM, Vijay  wrote:

> What is the version you are using? is it Multi DC setup? Are you seeing a
> lot of dropped Mutations/Messages? Are the nodes going up and down all the
> time while the repair is running?
>
> Regards,
> 
>
>
>
>
> On Tue, May 8, 2012 at 2:05 PM, Bill Au  wrote:
>
>> There are no error message in my log.
>>
>> I ended up restarting all the nodes in my cluster.  After that I was able
>> to run repair successfully on one of the node.  It took about 40 minutes.
>> Feeling lucky I ran repair on another node and it is stuck again.
>>
>> tpstats show 1 active and 1 pending AntiEntropySessions.  netstats and
>> compactionstats show no activity.  I took a close look at the log file, it
>> shows that the node requested merkle tree from 4 nodes (including itself).
>> It actually received 3 of those merkle trees.  It looks like it is stuck
>> waiting for that last one.  I checked the node where the request was sent
>> to, there isn't anything in the log on repair.  So it looks like the merkle
>> tree request has gotten lost some how.  It has been 8 hours since the
>> repair was issue and it is still stuck.  I am going to let it run a bit
>> longer to see if it will eventually finish.
>>
>> I have observed that if I restart all the nodes, I would be able to run
>> repair successfully on a single node.  I have done that twice already.  But
>> after that all repairs will hang.  Since we are supposed to run repair
>> periodically, having to restart all nodes before running repair on each
>> node isn't really viable for us.
>>
>> Bill
>>
>>
>> On Tue, May 8, 2012 at 6:04 AM, aaron morton wrote:
>>
>>> When you look in the logs please let me know if you see this error…
>>> https://issues.apache.org/jira/browse/CASSANDRA-4223
>>>
>>> I look at nodetool compactionstats (for the Merkle tree phase),
>>>  nodetool netstats for the streaming, and this to check for streaming
>>> progress:
>>>
>>> while true; do date; diff <(nodetool -h localhost netstats) <(sleep 5 &&
>>> nodetool -h localhost netstats); done
>>>
>>> Or use Data Stax Ops Centre where possible
>>> http://www.datastax.com/products/opscenter
>>>
>>> Cheers
>>>
>>>
>>>   -
>>> Aaron Morton
>>> Freelance Developer
>>> @aaronmorton
>>> http://www.thelastpickle.com
>>>
>>> On 8/05/2012, at 2:15 PM, Ben Coverston wrote:
>>>
>>> Check the log files for warnings or errors. They may indicate why your
>>> repair failed.
>>>
>>> On Mon, May 7, 2012 at 10:09 AM, Bill Au  wrote:
>>>
 I restarted the nodes and then restarted the repair.  It is still
 hanging like before.  Do I keep repeating until the repair actually finish?

 Bill


 On Fri, May 4, 2012 at 2:18 PM, Rob Coli  wrote:

> On Fri, May 4, 2012 at 10:30 AM, Bill Au  wrote:
> > I know repair may take a long time to run.  I am running repair on a
> node
> > with about 15 GB of data and it is taking more than 24 hours.  Is
> that
> > normal?  Is there any way to get status of the repair?  tpstats does
> show 2
> > active and 2 pending AntiEntropySessions.  But netstats and
> compactionstats
> > show no activity.
>
> As indicated by various recent threads to this effect, many versions
> of cassandra (including current 1.0.x release) contain bugs which
> sometimes prevent repair from completing. The other threads suggest
> that some of these bugs result in the state you are in now, where you
> do not see anything that looks like appropriate activity.
> Unfortunately the only solution offered on these other threads is the
> one I will now offer, which is to restart the participating nodes and
> re-start the repair. I am unaware of any JIRA tickets tracking these
> bugs (which doesn't mean they don't exist, of course) so you might
> want to file one. :)
>
> =Rob
>
> --
> =Robert Coli
> AIM>ALK - rc...@palominodb.com
> YAHOO - rcoli.palominob
> SKYPE - rcoli_palominodb
>


>>>
>>>
>>> --
>>> Ben Coverston
>>> DataStax -- The Apache Cassandra Company
>>>
>>>
>>>
>>
>


Re: Keyspace lost after restart

2012-05-09 Thread Conan Cook
Sorry, forgot to mention we're running Cassandra 1.1.

Conan

On 8 May 2012 17:51, Conan Cook  wrote:

> Hi Cassandra Folk,
>
> We've experienced a problem a couple of times where Cassandra nodes lose a
> keyspace after a restart.  We've restarted 2 out of 3 nodes, and they have
> both experienced this problem; clearly we're doing something wrong, but
> don't know what.  The data files are all still there, as before, but the
> node can't see the keyspace (we only have one).  Tthe nodetool still says
> that each one is responsible for 33% of the keys, but the disk usage has
> dropped to a tiny amount on the nodes that we've restarted.  I saw this:
>
>
> http://mail-archives.apache.org/mod_mbox/cassandra-user/201202.mbox/%3c4f3582e7.20...@conga.com%3E
>
> Seems to be exactly our problem, but we have not modified the
> cassandra.yaml - we have overwritten it through an automated process, and
> that happened just before restarting, but the contents did not change.
>
> Any ideas as to what might cause this, or how the keyspace can be restored
> (like I say, the data is all still in the data directory).
>
> We're running in AWS.
>
> Thanks,
>
>
> Conan
>


RE: sstableloader 1.1 won't stream

2012-05-09 Thread Pieter Callewaert
I don't see any entries in the logs of the nodes.

I've disabled SELinux, to be sure this wasn't a blocking factor, and tried 
adding -Djava.net.preferIPv4Stack=true to bin/sstableloader, but no change 
unfortunately.

To summarize, I'm trying to use sstableloader from a server (CentOS release 5.8 
(Final)) not running Cassandra to a 3-node Cassandra cluster. All running 1.1.
My next step will be to try to use sstableloader on one of the nodes from the 
cluster, to see if that works...

If anyone has any other ideas, please share.

Kind regards,
Pieter Callewaert

-Original Message-
From: Sylvain Lebresne [mailto:sylv...@datastax.com] 
Sent: woensdag 9 mei 2012 10:45
To: user@cassandra.apache.org
Subject: Re: sstableloader 1.1 won't stream

Have you checked for errors in the servers' logs?

--
Sylvain

On Tue, May 8, 2012 at 1:24 PM, Pieter Callewaert 
 wrote:
> I've updated all nodes to 1.1 but I keep getting the same problem...
> Any other thoughts about this?
>
> Kind regards,
> Pieter
>
> -Original Message-
> From: Benoit Perroud [mailto:ben...@noisette.ch]
> Sent: maandag 7 mei 2012 22:21
> To: user@cassandra.apache.org
> Subject: Re: sstableloader 1.1 won't stream
>
> You may want to upgrade all your nodes to 1.1.
>
> The streaming process connect to every living nodes of the cluster (you can 
> explicitely diable some nodes), so all nodes need to speak 1.1.
>
>
>
> 2012/5/7 Pieter Callewaert :
>> Hi,
>>
>>
>>
>> I'm trying to upgrade our bulk load process in our testing env.
>>
>> We use the SSTableSimpleUnsortedWriter to write tables, and use 
>> sstableloader to stream it into our cluster.
>>
>> I've changed the writer program to fit to the 1.1 api, but now I'm 
>> having troubles to load them to our cluster. The cluster exists out 
>> of one 1.1 node and two 1.0.9 nodes.
>>
>>
>>
>> I've enabled debug as parameter and in the log4j conf.
>>
>>
>>
>> [root@bms-app1 ~]# ./apache-cassandra/bin/sstableloader --debug -d
>> 10.10.10.100 /tmp/201205071234/MapData024/HOS/
>>
>> INFO 16:25:40,735 Opening
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1 (1588949 bytes)
>>
>> INFO 16:25:40,755 JNA not found. Native methods will be disabled.
>>
>> DEBUG 16:25:41,060 INDEX LOAD TIME for
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1: 327 ms.
>>
>> Streaming revelant part of
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to 
>> [/10.10.10.102, /10.10.10.100, /10.10.10.101]
>>
>> INFO 16:25:41,083 Stream context metadata 
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,084 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,087 Streaming to /10.10.10.102
>>
>> DEBUG 16:25:41,092 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%
>>
>> INFO 16:25:41,099 Stream context metadata 
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,100 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,100 Streaming to /10.10.10.100
>>
>> DEBUG 16:25:41,100 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%
>>
>> INFO 16:25:41,102 Stream context metadata 
>> [/tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%], 1 sstables.
>>
>> DEBUG 16:25:41,102 Adding file
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db to be streamed.
>>
>> INFO 16:25:41,102 Streaming to /10.10.10.101
>>
>> DEBUG 16:25:41,102 Files are
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%
>>
>>
>>
>> progress: [/10.10.10.102 0/1 (0)] [/10.10.10.100 0/1 (0)]
>> [/10.10.10.101 0/1 (0)] [total: 0 - 0MB/s (avg: 0MB/s)] WARN
>> 16:25:41,107 Failed attempt 1 to connect to /10.10.10.101 to stream 
>> /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=2
>> progress=0/6566400 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:41,108 Failed attempt 1 to connect to /10.10.10.102 to 
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6557280 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> WARN 16:25:41,108 Failed attempt 1 to connect to /10.10.10.100 to 
>> stream /tmp/201205071234/MapData024/HOS/MapData024-HOS-hc-1-Data.db
>> sections=1
>> progress=0/6551840 - 0%. Retrying in 4000 ms. (java.net.SocketException:
>> Invalid argument or cannot assign requested address)
>>
>> progress: [/10.10.10.102 0/1 (0)] [/10.10.10.100 0/1 (0)]
>> [/10.10.10.101 0/1 (0)] [total: 0 - 0MB/s (av

Re: getting status of long running repair

2012-05-09 Thread Vijay
Are you by using Broadcast Address? if yes then you might be affected by
https://issues.apache.org/jira/browse/CASSANDRA-3503

>>> Nodes are all up while repairing is running.
I should have been clear are you seeing the following messages in logs
(UP/DOWN) during the period of the repair...
 INFO [GossipStage:1] 2012-05-01 19:52:00,515 Gossiper.java (line 804)
InetAddress /xx.xx.xx.xx is now UP


Regards,




On Wed, May 9, 2012 at 5:49 AM, Bill Au  wrote:

> I am running 1.0.8.  Two data center with 8 machines in each dc.  Nodes
> are all up while repairing is running.  No dropped Mutations/Messages.  I
> do see HintedHandoff messages.
>
> Bill
>
>
> On Tue, May 8, 2012 at 11:15 PM, Vijay  wrote:
>
>> What is the version you are using? is it Multi DC setup? Are you seeing a
>> lot of dropped Mutations/Messages? Are the nodes going up and down all the
>> time while the repair is running?
>>
>> Regards,
>> 
>>
>>
>>
>>
>> On Tue, May 8, 2012 at 2:05 PM, Bill Au  wrote:
>>
>>> There are no error message in my log.
>>>
>>> I ended up restarting all the nodes in my cluster.  After that I was
>>> able to run repair successfully on one of the node.  It took about 40
>>> minutes.  Feeling lucky I ran repair on another node and it is stuck again.
>>>
>>> tpstats show 1 active and 1 pending AntiEntropySessions.  netstats and
>>> compactionstats show no activity.  I took a close look at the log file, it
>>> shows that the node requested merkle tree from 4 nodes (including itself).
>>> It actually received 3 of those merkle trees.  It looks like it is stuck
>>> waiting for that last one.  I checked the node where the request was sent
>>> to, there isn't anything in the log on repair.  So it looks like the merkle
>>> tree request has gotten lost some how.  It has been 8 hours since the
>>> repair was issue and it is still stuck.  I am going to let it run a bit
>>> longer to see if it will eventually finish.
>>>
>>> I have observed that if I restart all the nodes, I would be able to run
>>> repair successfully on a single node.  I have done that twice already.  But
>>> after that all repairs will hang.  Since we are supposed to run repair
>>> periodically, having to restart all nodes before running repair on each
>>> node isn't really viable for us.
>>>
>>> Bill
>>>
>>>
>>> On Tue, May 8, 2012 at 6:04 AM, aaron morton wrote:
>>>
 When you look in the logs please let me know if you see this error…
 https://issues.apache.org/jira/browse/CASSANDRA-4223

 I look at nodetool compactionstats (for the Merkle tree phase),
  nodetool netstats for the streaming, and this to check for streaming
 progress:

 while true; do date; diff <(nodetool -h localhost netstats) <(sleep 5
 && nodetool -h localhost netstats); done

 Or use Data Stax Ops Centre where possible
 http://www.datastax.com/products/opscenter

 Cheers


   -
 Aaron Morton
 Freelance Developer
 @aaronmorton
 http://www.thelastpickle.com

 On 8/05/2012, at 2:15 PM, Ben Coverston wrote:

 Check the log files for warnings or errors. They may indicate why your
 repair failed.

 On Mon, May 7, 2012 at 10:09 AM, Bill Au  wrote:

> I restarted the nodes and then restarted the repair.  It is still
> hanging like before.  Do I keep repeating until the repair actually 
> finish?
>
> Bill
>
>
> On Fri, May 4, 2012 at 2:18 PM, Rob Coli  wrote:
>
>> On Fri, May 4, 2012 at 10:30 AM, Bill Au  wrote:
>> > I know repair may take a long time to run.  I am running repair on
>> a node
>> > with about 15 GB of data and it is taking more than 24 hours.  Is
>> that
>> > normal?  Is there any way to get status of the repair?  tpstats
>> does show 2
>> > active and 2 pending AntiEntropySessions.  But netstats and
>> compactionstats
>> > show no activity.
>>
>> As indicated by various recent threads to this effect, many versions
>> of cassandra (including current 1.0.x release) contain bugs which
>> sometimes prevent repair from completing. The other threads suggest
>> that some of these bugs result in the state you are in now, where you
>> do not see anything that looks like appropriate activity.
>> Unfortunately the only solution offered on these other threads is the
>> one I will now offer, which is to restart the participating nodes and
>> re-start the repair. I am unaware of any JIRA tickets tracking these
>> bugs (which doesn't mean they don't exist, of course) so you might
>> want to file one. :)
>>
>> =Rob
>>
>> --
>> =Robert Coli
>> AIM>ALK - rc...@palominodb.com
>> YAHOO - rcoli.palominob
>> SKYPE - rcoli_palominodb
>>
>
>


 --
 Ben Coverston
 DataStax -- The Apache Cassandra Company



>>>
>>
>


Re: enforcing ordering

2012-05-09 Thread Franc Carter
On Tue, May 8, 2012 at 8:21 PM, Franc Carter wrote:

> On Tue, May 8, 2012 at 8:09 PM, aaron morton wrote:
>
>> Can you store the corrections in a separate CF?
>>
>
We sat down and thought about this harder - it looks like a good solution
for us that may makel other hard problems go away - thanks.

cheers


> Yes, I thought of that, but that turns on read in to two ;-(
>
>
>>
>> When the client reads the key, reads from the original the corrects CF at
>> the same time. Apply the correction only on the client side.
>>
>> When you have confirmed the ingest has completed, run a background jobs
>> to apply the corrections, store the updated values and delete the
>> correction data.
>>
>
> I was thinking down this path, but I ended up chasing the rabbit down a
> deep hole of race conditions . . .
>
> cheers
>
>
>>
>> Cheers
>>
>>   -
>> Aaron Morton
>> Freelance Developer
>> @aaronmorton
>> http://www.thelastpickle.com
>>
>> On 8/05/2012, at 9:35 PM, Franc Carter wrote:
>>
>>
>> Hi,
>>
>> I'm wondering if there is a common 'pattern' to address a scenario we
>> will have to deal with.
>>
>> We will be storing a set of Column/Value pairs per Key where the
>> Column/Values are read from a set of files that we download regularly. We
>> need the loading to be resilient and we can receive corrections for some of
>> the Column/Values that can only be loaded after the initial data has been
>> inserted.
>>
>> The challenge we have is that we have a strong preference for
>> active/active loading of data and can't see how to achieve this without
>> some form of serialisation (which Cassandra doesn't support - correct ?)
>>
>> thanks
>>
>> --
>> *Franc Carter* | Systems architect | Sirca Ltd
>>  
>> franc.car...@sirca.org.au | www.sirca.org.au
>> Tel: +61 2 9236 9118
>>  Level 9, 80 Clarence St, Sydney NSW 2000
>> PO Box H58, Australia Square, Sydney NSW 1215
>>
>>
>>
>
>
> --
>
> *Franc Carter* | Systems architect | Sirca Ltd
>  
>
> franc.car...@sirca.org.au | www.sirca.org.au
>
> Tel: +61 2 9236 9118
>
> Level 9, 80 Clarence St, Sydney NSW 2000
>
> PO Box H58, Australia Square, Sydney NSW 1215
>
>


-- 

*Franc Carter* | Systems architect | Sirca Ltd
 

franc.car...@sirca.org.au | www.sirca.org.au

Tel: +61 2 9236 9118

Level 9, 80 Clarence St, Sydney NSW 2000

PO Box H58, Australia Square, Sydney NSW 1215


Re: Error deleting column families with 1.1

2012-05-09 Thread Data Craftsman
We have similar issue. I'll try to reproduce it and upload logs soon.

On Wed, May 9, 2012 at 3:30 AM, André Cruz  wrote:
> Here it is: https://issues.apache.org/jira/browse/CASSANDRA-4230
>
> Please let me know if you need further info.
>
> Best regards,
> André
>
> On May 8, 2012, at 23:55 , aaron morton wrote:
>
> Could you please create a ticket for this
> on https://issues.apache.org/jira/browse/CASSANDRA
>
> Please include:
> * operating system
> * keyspace / column family definition
> * output of of "ls -lah" for the "/var/lib/cassandra/data/Disco/Client/"
> directory after the error occurs.
>
> Thanks
>
> -
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
>
> On 8/05/2012, at 9:00 AM, André Cruz wrote:
>
> Hello.
>
> Since I upgraded to Cassandra 1.1, I get the following error when trying to
> delete a CF. After this happens the CF is not accessible anymore, but I
> cannot create another one with the same name until I restart the server.
>
> INFO [MigrationStage:1] 2012-05-07 18:10:12,682 ColumnFamilyStore.java (line
> 634) Enqueuing flush of Memtable-schema_columnfamilies@1128094887(978/1222
> serialized/live bytes, 21 ops)
> INFO [FlushWriter:2] 2012-05-07 18:10:12,682 Memtable.java (line 266)
> Writing Memtable-schema_columnfamilies@1128094887(978/1222 serialized/live
> bytes, 21 ops)
> INFO [FlushWriter:2] 2012-05-07 18:10:12,720 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-28-Data.db
> (1041 bytes)
> INFO [MigrationStage:1] 2012-05-07 18:10:12,721 ColumnFamilyStore.java (line
> 634) Enqueuing flush of Memtable-schema_columns@1599271050(392/490
> serialized/live bytes, 8 ops)
> INFO [FlushWriter:2] 2012-05-07 18:10:12,722 Memtable.java (line 266)
> Writing Memtable-schema_columns@1599271050(392/490 serialized/live bytes, 8
> ops)
> INFO [CompactionExecutor:8] 2012-05-07 18:10:12,722 CompactionTask.java
> (line 114) Compacting
> [SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-26-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-28-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfam
> ilies-hc-27-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-25-Data.db')]
> INFO [FlushWriter:2] 2012-05-07 18:10:12,806 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/system/schema_columns/system-schema_columns-hc-23-Data.db
> (447 bytes)
> INFO [CompactionExecutor:8] 2012-05-07 18:10:12,811 CompactionTask.java
> (line 225) Compacted to
> [/var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-hc-29-Data.db,].
>  24,797 to 21,431
> (~86% of original) bytes for 2 keys at 0.232252MB/s.  Time: 88ms.
> ERROR [MigrationStage:1] 2012-05-07 18:10:12,895 CLibrary.java (line 158)
> Unable to create hard link
> com.sun.jna.LastErrorException: errno was 17
> at org.apache.cassandra.utils.CLibrary.link(Native Method)
> at org.apache.cassandra.utils.CLibrary.createHardLink(CLibrary.java:150)
> at
> org.apache.cassandra.db.Directories.snapshotLeveledManifest(Directories.java:343)
> at
> org.apache.cassandra.db.ColumnFamilyStore.snapshotWithoutFlush(ColumnFamilyStore.java:1450)
> at
> org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:1483)
> at org.apache.cassandra.db.DefsTable.dropColumnFamily(DefsTable.java:512)
> at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:403)
> at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:270)
> at
> org.apache.cassandra.service.MigrationManager$1.call(MigrationManager.java:214)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> ERROR [Thrift:17] 2012-05-07 18:10:12,898 CustomTThreadPoolServer.java (line
> 204) Error occurred during processing of message.
> java.lang.RuntimeException: java.util.concurrent.ExecutionException:
> java.io.IOError: java.io.IOException: Unable to create hard link from
> /var/lib/cassandra/data/Disco/Client/Client.json to /var/lib/cassandra/data/
> Disco/Client/snapshots/1336410612893-Client/Client.json (errno 17)
> at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:372)
> at
> org.apache.cassandra.service.MigrationManager.announce(MigrationManager.java:191)
> at
> org.apache.cassandra.service.MigrationManager.announceColumnFamilyDrop(MigrationManager.java:182)
> at
> org.apache.cassandra.thrift.CassandraServer.system_drop_column_family(CassandraServer.jav