Re: Cassandra restart
I will try to reproduce problem on smaller test cluster. It was rather easy, cluster contains 4 servers. Log's fragment from restarted node (10.2.3.38): DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 CassandraServer.java (line 214) get_slice DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 StorageProxy.java (line 239) weakreadlocal reading SliceFromReadCommand(table='Keyspace1', key='00849706', column_parent='QueryPath(columnFamilyName='Super1', superColumnName='[...@6ca50fbe', columnName='null')', start='1', finish='0', reversed=true, count=2) DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 StorageProxy.java (line 251) weakreadremote reading SliceFromReadCommand(table='Keyspace1', key='00849706', column_parent='QueryPath(columnFamilyName='Super1', superColumnName='[...@6ca50fbe', columnName='null')', start='1', finish='0', reversed=true, count=2) from 207...@10.3.2.40:7000 ... ERROR [pool-1-thread-64] 2009-10-15 14:18:21,281 Cassandra.java (line 679) Internal error processing get_slice java.lang.RuntimeException: error reading key 00849706 at org.apache.cassandra.service.StorageProxy.weakReadRemote(StorageProxy.java:265) at org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:312) at org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:95) at org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:177) at org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:252) at org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:215) at org.apache.cassandra.service.Cassandra$Processor$get_slice.process(Cassandra.java:671) at org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:627) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) Caused by: java.util.concurrent.TimeoutException: Operation timed out. at org.apache.cassandra.net.AsyncResult.get(AsyncResult.java:97) at org.apache.cassandra.service.StorageProxy.weakReadRemote(StorageProxy.java:261) ... 11 more Log's fragment from 10.2.3.40: DEBUG [ROW-READ-STAGE:4] 2009-10-15 14:18:16,308 ReadVerbHandler.java (line 100) Read key 00849706; sending response to 207...@10.3.2.38:7000 DEBUG [CONSISTENCY-MANAGER:2] 2009-10-15 14:18:16,308 ConsistencyManager.java (line 168) Reading consistency digest for 00849706 from 527...@[10.3.2.39:7000, 10.3.2.41:7000] I have full logs, but they are about half of gigabyte for each node. If it's needed I can put them somewhere accessible by http. How to reproduce: - configure cluster for 4 nodes, changes in storage-conf.xml: 3 8 16 - edit attached scripts with correct node's IPs - run perl writecluster.pl -c 8 and wait for 10-20 minutes - run perl readcluster.pl - look at error :) -- Teodor Sigaev E-mail: teo...@sigaev.ru WWW: http://www.sigaev.ru/ writecluster.pl Description: Perl program readcluster.pl Description: Perl program
Re: Cassandra restart
Does this still happen after a few dozen queries to 10.2.3.38? It looks like .40 is finding the key and trying to send it back, so one possibility is that .40's pooled socket to .38 hasn't realized that .38 was restarted, so the first few messages might get lost before it reconnects. 2009/10/15 Teodor Sigaev : >> I will try to reproduce problem on smaller test cluster. > > It was rather easy, cluster contains 4 servers. > Log's fragment from restarted node (10.2.3.38): > > DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 CassandraServer.java (line > 214) get_slice > DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 StorageProxy.java (line > 239) weakreadlocal reading SliceFromReadCommand(table='Keyspace1', > key='00849706', > column_parent='QueryPath(columnFamilyName='Super1', > superColumnName='[...@6ca50fbe', columnName='null')', start='1', finish='0', > reversed=true, count=2) > DEBUG [pool-1-thread-64] 2009-10-15 14:18:16,290 StorageProxy.java (line > 251) weakreadremote reading SliceFromReadCommand(table='Keyspace1', > key='00849706', > column_parent='QueryPath(columnFamilyName='Super1', > superColumnName='[...@6ca50fbe', columnName='null')', start='1', finish='0', > reversed=true, count=2) from 207...@10.3.2.40:7000 > ... > ERROR [pool-1-thread-64] 2009-10-15 14:18:21,281 Cassandra.java (line 679) > Internal error processing get_slice > java.lang.RuntimeException: error reading key > 00849706 > at > org.apache.cassandra.service.StorageProxy.weakReadRemote(StorageProxy.java:265) > at > org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:312) > at > org.apache.cassandra.service.CassandraServer.readColumnFamily(CassandraServer.java:95) > at > org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:177) > at > org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:252) > at > org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:215) > at > org.apache.cassandra.service.Cassandra$Processor$get_slice.process(Cassandra.java:671) > at > org.apache.cassandra.service.Cassandra$Processor.process(Cassandra.java:627) > at > org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) > at java.lang.Thread.run(Thread.java:636) > Caused by: java.util.concurrent.TimeoutException: Operation timed out. > at org.apache.cassandra.net.AsyncResult.get(AsyncResult.java:97) > at > org.apache.cassandra.service.StorageProxy.weakReadRemote(StorageProxy.java:261) > ... 11 more > > Log's fragment from 10.2.3.40: > DEBUG [ROW-READ-STAGE:4] 2009-10-15 14:18:16,308 ReadVerbHandler.java (line > 100) Read key 00849706; sending response to > 207...@10.3.2.38:7000 > > DEBUG [CONSISTENCY-MANAGER:2] 2009-10-15 14:18:16,308 > ConsistencyManager.java (line 168) Reading consistency digest for > 00849706 from 527...@[10.3.2.39:7000, > 10.3.2.41:7000] > > I have full logs, but they are about half of gigabyte for each node. If it's > needed I can put them somewhere accessible by http. > > How to reproduce: > - configure cluster for 4 nodes, changes in storage-conf.xml: > 3 > 8 > 16 > - edit attached scripts with correct node's IPs > - run perl writecluster.pl -c 8 and wait for 10-20 minutes > - run perl readcluster.pl > - look at error :) > > -- > Teodor Sigaev E-mail: teo...@sigaev.ru > WWW: http://www.sigaev.ru/ >
Re: Cassandra restart
Does this still happen after a few dozen queries to 10.2.3.38? Hmm. it depends of downtime and how long it worked before restart. If both times is rather small then errors/timeouts go away after ten minutes. If at least one of that times is greater than half an hour then full restoration could take more than several hours. It looks like .40 is finding the key and trying to send it back, so one possibility is that .40's pooled socket to .38 hasn't realized that .38 was restarted, so the first few messages might get lost before it reconnects. Downtime was long enough, all live nodes say by 'nodeprobe cluster' that .38 is down. It works well after restarting whole cluster. -- Teodor Sigaev E-mail: teo...@sigaev.ru WWW: http://www.sigaev.ru/
Thrift Perl API Timeout Issues
Using the Thrift Perl API into Cassandra, I am running into what is endearingly referred to as the 4 bytes of doom: TSocket: timed out reading 4 bytes from localhost:9160 The script I am using is fairly simple. I have a text file that has about 3.6 million lines that are formatted like: f...@bar.com 1234 The Cassandra dataset is a single column family called Users in the Mailings keyspace with a data layout of: Users = { 'f...@example.com': { email: 'f...@example.com', person_id: '123456', send_dates_2009-09-30: '2245', send_dates_2009-10-01: '2247', }, } There are about 3.5 million rows in the Users column family and each row has no more than 4 columns (listed above). Some only have 3 (one of the send_dates_-MM-DD isn't there). The script parses it and then connects to Cassandra and does a get_slice and counts the return values adding that to a hash: my ($value) = $client->get_slice( 'Mailings', $email, Cassandra::ColumnParent->new({ column_family => 'Users', }), Cassandra::SlicePredicate->new({ slice_range => Cassandra::SliceRange->new({ start => 'send_dates_2009-09-29', finish => 'send_dates_2009-10-30', }), }), Cassandra::ConsistencyLevel::ONE ); $counter{($#{$value} + 1)}++; For the most part, this script times out after 1 minute or so. Replacing the get_slice with a get_count, I can get it to about 2 million queries before I get the timeout. Replacing the get_slice with a get, I make it to about 2.5 million before I get the timeout. The only way I could get it to run all the way through was to add a 1/100 of a second sleep during every iteration. I was able to get the script to complete when I shut down everything else on the machine (and it took 177m to complete). But since this is a semi-production machine, I had to turn everything back on afterwards. So for poops and laughs (at the recommendation of jbellis), I rewrote the script in Python and it has since run (using get_slice) 3 times fully without timing out (approximately 130m in Python) with everything else running on the machine. My question is, having seen this same thing in the PHP API and it is my understanding that the Perl API was based on the PHP API, could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here too? Is anyone else seeing this issue? If so, have you gotten around it? Thanks. -e
Re: Thrift Perl API Timeout Issues
You need to call $socket->setRecvTimeout() With a higher number in ms. On Oct 15, 2009, at 11:26 AM, Eric Lubow wrote: Using the Thrift Perl API into Cassandra, I am running into what is endearingly referred to as the 4 bytes of doom: TSocket: timed out reading 4 bytes from localhost:9160 The script I am using is fairly simple. I have a text file that has about 3.6 million lines that are formatted like: f...@bar.com 1234 The Cassandra dataset is a single column family called Users in the Mailings keyspace with a data layout of: Users = { 'f...@example.com': { email: 'f...@example.com', person_id: '123456', send_dates_2009-09-30: '2245', send_dates_2009-10-01: '2247', }, } There are about 3.5 million rows in the Users column family and each row has no more than 4 columns (listed above). Some only have 3 (one of the send_dates_-MM-DD isn't there). The script parses it and then connects to Cassandra and does a get_slice and counts the return values adding that to a hash: my ($value) = $client->get_slice( 'Mailings', $email, Cassandra::ColumnParent->new({ column_family => 'Users', }), Cassandra::SlicePredicate->new({ slice_range => Cassandra::SliceRange->new({ start => 'send_dates_2009-09-29', finish => 'send_dates_2009-10-30', }), }), Cassandra::ConsistencyLevel::ONE ); $counter{($#{$value} + 1)}++; For the most part, this script times out after 1 minute or so. Replacing the get_slice with a get_count, I can get it to about 2 million queries before I get the timeout. Replacing the get_slice with a get, I make it to about 2.5 million before I get the timeout. The only way I could get it to run all the way through was to add a 1/100 of a second sleep during every iteration. I was able to get the script to complete when I shut down everything else on the machine (and it took 177m to complete). But since this is a semi-production machine, I had to turn everything back on afterwards. So for poops and laughs (at the recommendation of jbellis), I rewrote the script in Python and it has since run (using get_slice) 3 times fully without timing out (approximately 130m in Python) with everything else running on the machine. My question is, having seen this same thing in the PHP API and it is my understanding that the Perl API was based on the PHP API, could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here too? Is anyone else seeing this issue? If so, have you gotten around it? Thanks. -e
Re: Thrift Perl API Timeout Issues
What is the default? On Thu, Oct 15, 2009 at 10:37 AM, Jake Luciani wrote: > You need to call > $socket->setRecvTimeout() > With a higher number in ms. > > > On Oct 15, 2009, at 11:26 AM, Eric Lubow wrote: > > Using the Thrift Perl API into Cassandra, I am running into what is > endearingly referred to as the 4 bytes of doom: > TSocket: timed out reading 4 bytes from localhost:9160 > The script I am using is fairly simple. I have a text file that has about > 3.6 million lines that are formatted like: ...@bar.com 1234 > The Cassandra dataset is a single column family called Users in the Mailings > keyspace with a data layout of: > Users = { > 'f...@example.com': { > email: 'f...@example.com', > person_id: '123456', > send_dates_2009-09-30: '2245', > send_dates_2009-10-01: '2247', > }, > } > There are about 3.5 million rows in the Users column family and each row has > no more than 4 columns (listed above). Some only have 3 (one of the > send_dates_-MM-DD isn't there). > The script parses it and then connects to Cassandra and does a get_slice and > counts the return values adding that to a hash: > my ($value) = $client->get_slice( > 'Mailings', > $email, > Cassandra::ColumnParent->new({ > column_family => 'Users', > }), > Cassandra::SlicePredicate->new({ > slice_range => Cassandra::SliceRange->new({ > start => 'send_dates_2009-09-29', > finish => 'send_dates_2009-10-30', > }), > }), > Cassandra::ConsistencyLevel::ONE > ); > $counter{($#{$value} + 1)}++; > For the most part, this script times out after 1 minute or so. Replacing the > get_slice with a get_count, I can get it to about 2 million queries before I > get the timeout. Replacing the get_slice with a get, I make it to about 2.5 > million before I get the timeout. The only way I could get it to run all > the way through was to add a 1/100 of a second sleep during every iteration. > I was able to get the script to complete when I shut down everything else > on the machine (and it took 177m to complete). But since this is a > semi-production machine, I had to turn everything back on afterwards. > So for poops and laughs (at the recommendation of jbellis), I rewrote the > script in Python and it has since run (using get_slice) 3 times fully > without timing out (approximately 130m in Python) with everything else > running on the machine. > My question is, having seen this same thing in the PHP API and it is my > understanding that the Perl API was based on the PHP API, > could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here > too? Is anyone else seeing this issue? If so, have you gotten around it? > Thanks. > -e
Re: Thrift Perl API Timeout Issues
I think it's 100ms. I need to increase it to match python I guess. Sent from my iPhone On Oct 15, 2009, at 11:40 AM, Jonathan Ellis wrote: What is the default? On Thu, Oct 15, 2009 at 10:37 AM, Jake Luciani wrote: You need to call $socket->setRecvTimeout() With a higher number in ms. On Oct 15, 2009, at 11:26 AM, Eric Lubow wrote: Using the Thrift Perl API into Cassandra, I am running into what is endearingly referred to as the 4 bytes of doom: TSocket: timed out reading 4 bytes from localhost:9160 The script I am using is fairly simple. I have a text file that has about 3.6 million lines that are formatted like: f...@bar.com 1234 The Cassandra dataset is a single column family called Users in the Mailings keyspace with a data layout of: Users = { 'f...@example.com': { email: 'f...@example.com', person_id: '123456', send_dates_2009-09-30: '2245', send_dates_2009-10-01: '2247', }, } There are about 3.5 million rows in the Users column family and each row has no more than 4 columns (listed above). Some only have 3 (one of the send_dates_-MM-DD isn't there). The script parses it and then connects to Cassandra and does a get_slice and counts the return values adding that to a hash: my ($value) = $client->get_slice( 'Mailings', $email, Cassandra::ColumnParent->new({ column_family => 'Users', }), Cassandra::SlicePredicate->new({ slice_range => Cassandra::SliceRange->new({ start => 'send_dates_2009-09-29', finish => 'send_dates_2009-10-30', }), }), Cassandra::ConsistencyLevel::ONE ); $counter{($#{$value} + 1)}++; For the most part, this script times out after 1 minute or so. Replacing the get_slice with a get_count, I can get it to about 2 million queries before I get the timeout. Replacing the get_slice with a get, I make it to about 2.5 million before I get the timeout. The only way I could get it to run all the way through was to add a 1/100 of a second sleep during every iteration. I was able to get the script to complete when I shut down everything else on the machine (and it took 177m to complete). But since this is a semi-production machine, I had to turn everything back on afterwards. So for poops and laughs (at the recommendation of jbellis), I rewrote the script in Python and it has since run (using get_slice) 3 times fully without timing out (approximately 130m in Python) with everything else running on the machine. My question is, having seen this same thing in the PHP API and it is my understanding that the Perl API was based on the PHP API, could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here too? Is anyone else seeing this issue? If so, have you gotten around it? Thanks. -e
Re: Thrift Perl API Timeout Issues
My connection section of the script is here: # Connect to the database my $socket = new Thrift::Socket('localhost',9160); $socket->setSendTimeout(2500); $socket->setRecvTimeout(7500); my $transport = new Thrift::BufferedTransport($socket,2048,2048); my $protocol = new Thrift::BinaryProtocol($transport); my $client = Cassandra::CassandraClient->new($protocol); I even tried it with combinations of 1024 as the size and 1000 as the SendTimeout and 5000 as the RecvTimeout. -e On Thu, Oct 15, 2009 at 11:42 AM, Jake Luciani wrote: > I think it's 100ms. I need to increase it to match python I guess. > > Sent from my iPhone > > > On Oct 15, 2009, at 11:40 AM, Jonathan Ellis wrote: > > What is the default? >> >> On Thu, Oct 15, 2009 at 10:37 AM, Jake Luciani wrote: >> >>> You need to call >>> $socket->setRecvTimeout() >>> With a higher number in ms. >>> >>> >>> On Oct 15, 2009, at 11:26 AM, Eric Lubow wrote: >>> >>> Using the Thrift Perl API into Cassandra, I am running into what is >>> endearingly referred to as the 4 bytes of doom: >>> TSocket: timed out reading 4 bytes from localhost:9160 >>> The script I am using is fairly simple. I have a text file that has >>> about >>> 3.6 million lines that are formatted like: f...@bar.com 1234 >>> The Cassandra dataset is a single column family called Users in the >>> Mailings >>> keyspace with a data layout of: >>> Users = { >>>'f...@example.com': { >>>email: 'f...@example.com', >>>person_id: '123456', >>>send_dates_2009-09-30: '2245', >>>send_dates_2009-10-01: '2247', >>>}, >>> } >>> There are about 3.5 million rows in the Users column family and each row >>> has >>> no more than 4 columns (listed above). Some only have 3 (one of the >>> send_dates_-MM-DD isn't there). >>> The script parses it and then connects to Cassandra and does a get_slice >>> and >>> counts the return values adding that to a hash: >>> my ($value) = $client->get_slice( >>> 'Mailings', >>> $email, >>> Cassandra::ColumnParent->new({ >>> column_family => 'Users', >>> }), >>> Cassandra::SlicePredicate->new({ >>> slice_range => Cassandra::SliceRange->new({ >>> start => 'send_dates_2009-09-29', >>> finish => 'send_dates_2009-10-30', >>> }), >>> }), >>> Cassandra::ConsistencyLevel::ONE >>> ); >>> $counter{($#{$value} + 1)}++; >>> For the most part, this script times out after 1 minute or so. Replacing >>> the >>> get_slice with a get_count, I can get it to about 2 million queries >>> before I >>> get the timeout. Replacing the get_slice with a get, I make it to about >>> 2.5 >>> million before I get the timeout. The only way I could get it to run all >>> the way through was to add a 1/100 of a second sleep during every >>> iteration. >>> I was able to get the script to complete when I shut down everything >>> else >>> on the machine (and it took 177m to complete). But since this is a >>> semi-production machine, I had to turn everything back on afterwards. >>> So for poops and laughs (at the recommendation of jbellis), I rewrote the >>> script in Python and it has since run (using get_slice) 3 times fully >>> without timing out (approximately 130m in Python) with everything else >>> running on the machine. >>> My question is, having seen this same thing in the PHP API and it is my >>> understanding that the Perl API was based on the PHP API, >>> could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here >>> too? Is anyone else seeing this issue? If so, have you gotten around >>> it? >>> Thanks. >>> -e >>> >>
Re: Thrift Perl API Timeout Issues
While on the topic, I'm using the python Thrift interface - if I wanted to, how would I change the timeout? I currently do: socket = TSocket.TSocket(host,port) If I wanted to change the timeout would I do something like: socket.setTimeout(timeout) or...? Sorry if I should be able to see this by looking at the code - I'm new to python. Thanks, Simon On Thu, Oct 15, 2009 at 11:42 AM, Jake Luciani wrote: > I think it's 100ms. I need to increase it to match python I guess. > > Sent from my iPhone > > On Oct 15, 2009, at 11:40 AM, Jonathan Ellis wrote: > >> What is the default? >> >> On Thu, Oct 15, 2009 at 10:37 AM, Jake Luciani wrote: >>> >>> You need to call >>> $socket->setRecvTimeout() >>> With a higher number in ms. >>> >>>
Re: Thrift Perl API Timeout Issues
On Thu, Oct 15, 2009 at 10:51 AM, Simon Smith wrote: > While on the topic, I'm using the python Thrift interface - if I > wanted to, how would I change the timeout? I currently do: > > socket = TSocket.TSocket(host,port) > > If I wanted to change the timeout would I do something like: > > socket.setTimeout(timeout) Yes, with timeout in ms. (The default appears to be "never timeout.") -Jonathan
Re: Thrift Perl API Timeout Issues
What happens if you set it to 10? On Oct 15, 2009, at 11:48 AM, Eric Lubow wrote: My connection section of the script is here: # Connect to the database my $socket = new Thrift::Socket('localhost',9160); $socket->setSendTimeout(2500); $socket->setRecvTimeout(7500); my $transport = new Thrift::BufferedTransport($socket,2048,2048); my $protocol = new Thrift::BinaryProtocol($transport); my $client = Cassandra::CassandraClient->new($protocol); I even tried it with combinations of 1024 as the size and 1000 as the SendTimeout and 5000 as the RecvTimeout. -e On Thu, Oct 15, 2009 at 11:42 AM, Jake Luciani wrote: I think it's 100ms. I need to increase it to match python I guess. Sent from my iPhone On Oct 15, 2009, at 11:40 AM, Jonathan Ellis wrote: What is the default? On Thu, Oct 15, 2009 at 10:37 AM, Jake Luciani wrote: You need to call $socket->setRecvTimeout() With a higher number in ms. On Oct 15, 2009, at 11:26 AM, Eric Lubow wrote: Using the Thrift Perl API into Cassandra, I am running into what is endearingly referred to as the 4 bytes of doom: TSocket: timed out reading 4 bytes from localhost:9160 The script I am using is fairly simple. I have a text file that has about 3.6 million lines that are formatted like: f...@bar.com 1234 The Cassandra dataset is a single column family called Users in the Mailings keyspace with a data layout of: Users = { 'f...@example.com': { email: 'f...@example.com', person_id: '123456', send_dates_2009-09-30: '2245', send_dates_2009-10-01: '2247', }, } There are about 3.5 million rows in the Users column family and each row has no more than 4 columns (listed above). Some only have 3 (one of the send_dates_-MM-DD isn't there). The script parses it and then connects to Cassandra and does a get_slice and counts the return values adding that to a hash: my ($value) = $client->get_slice( 'Mailings', $email, Cassandra::ColumnParent->new({ column_family => 'Users', }), Cassandra::SlicePredicate->new({ slice_range => Cassandra::SliceRange->new({ start => 'send_dates_2009-09-29', finish => 'send_dates_2009-10-30', }), }), Cassandra::ConsistencyLevel::ONE ); $counter{($#{$value} + 1)}++; For the most part, this script times out after 1 minute or so. Replacing the get_slice with a get_count, I can get it to about 2 million queries before I get the timeout. Replacing the get_slice with a get, I make it to about 2.5 million before I get the timeout. The only way I could get it to run all the way through was to add a 1/100 of a second sleep during every iteration. I was able to get the script to complete when I shut down everything else on the machine (and it took 177m to complete). But since this is a semi-production machine, I had to turn everything back on afterwards. So for poops and laughs (at the recommendation of jbellis), I rewrote the script in Python and it has since run (using get_slice) 3 times fully without timing out (approximately 130m in Python) with everything else running on the machine. My question is, having seen this same thing in the PHP API and it is my understanding that the Perl API was based on the PHP API, could http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here too? Is anyone else seeing this issue? If so, have you gotten around it? Thanks. -e
cassandra fatal error - "The name should match the name of the current column or super column"
I'm using the cassandra 0.4 release. I was loading a bunch of data into cassandra when the thrift api started throwing UnavailableExceptions. Checking the logs, I found errors that looked like the following: ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 DebuggableThreadPoolExecutor.java (line 85) Error in ThreadPoolExecutor java.lang.IllegalArgumentException: The name should match the name of the current column or super column at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) at org.apache.cassandra.db.Memtable.put(Memtable.java:139) at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) at org.apache.cassandra.db.Table.apply(Table.java:608) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 CassandraDaemon.java (line 71) Fatal exception in thread Thread[ROW-MUTATION-STAGE:2935,5,main] java.lang.IllegalArgumentException: The name should match the name of the current column or super column at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) at org.apache.cassandra.db.Memtable.put(Memtable.java:139) at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) at org.apache.cassandra.db.Table.apply(Table.java:608) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) Stopping and starting the cluster gives me something similar: ERROR - Error in executor futuretask java.util.concurrent.ExecutionException: java.lang.IllegalArgumentException: The name should match the name of the current column or super column at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) Caused by: java.lang.IllegalArgumentException: The name should match the name of the current column or super column at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) at org.apache.cassandra.db.ColumnFamily.resolve(ColumnFamily.java:408) at org.apache.cassandra.db.ColumnFamilyStore.merge(ColumnFamilyStore.java:477) at org.apache.cassandra.db.ColumnFamilyStore.doFileCompaction(ColumnFamilyStore.java:1078) at org.apache.cassandra.db.ColumnFamilyStore.doCompaction(ColumnFamilyStore.java:689) at org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:165) at org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:162) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:98) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:207) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) ... 2 more I haven't yet tried to repro, but was wondering if anyone had any insight on
Re: cassandra fatal error - "The name should match the name of the current column or super column"
The exception on restart is occurring during a compaction of already-written sstables. It logs what the files are beforehand ("Compacting [...]"). Could you gzip up those sstables and send those to me somehow (the Data, Index, and Filter files) along with the ColumnFamily definition from your config so I can try some tests out? -Jonathan On Thu, Oct 15, 2009 at 12:51 PM, Edmond Lau wrote: > I'm using the cassandra 0.4 release. I was loading a bunch of data > into cassandra when the thrift api started throwing > UnavailableExceptions. Checking the logs, I found errors that looked > like the following: > > ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 > DebuggableThreadPoolExecutor.java (line 85) Error in > ThreadPoolExecutor > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) > at > org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) > at > org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) > at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) > at org.apache.cassandra.db.Memtable.put(Memtable.java:139) > at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) > at org.apache.cassandra.db.Table.apply(Table.java:608) > at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) > at > org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 > CassandraDaemon.java (line 71) Fatal exception in thread > Thread[ROW-MUTATION-STAGE:2935,5,main] > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) > at > org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) > at > org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) > at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) > at org.apache.cassandra.db.Memtable.put(Memtable.java:139) > at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) > at org.apache.cassandra.db.Table.apply(Table.java:608) > at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) > at > org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > > Stopping and starting the cluster gives me something similar: > > ERROR - Error in executor futuretask > java.util.concurrent.ExecutionException: > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) > at java.util.concurrent.FutureTask.get(FutureTask.java:83) > at > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > Caused by: java.lang.IllegalArgumentException: The name should match > the name of the current column or super column > at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) > at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) > at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) > at org.apache.cassandra.db.ColumnFamily.resolve(ColumnFamily.java:408) > at org.apache.cassandra.db.ColumnFamilyStore.merge(ColumnFamilyStore.java:477) > at > org.apache.cassandra.db.ColumnFamilyStore.doFileCompaction(ColumnFamilyStore.java:1078) > at > org.apache.cassandra.db.ColumnFamilyStore.doCompaction(ColumnFamilyStore.java:689) > at > org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:165) > at > org.apache.cassandra.db.MinorCompactionManager$1.call(MinorCompactionManager.java:16
Re: cassandra fatal error - "The name should match the name of the current column or super column"
Quicker question: Is this ColumnFamily using UTF8Type as its comparator? On Thu, Oct 15, 2009 at 1:17 PM, Jonathan Ellis wrote: > The exception on restart is occurring during a compaction of > already-written sstables. It logs what the files are beforehand > ("Compacting [...]"). Could you gzip up those sstables and send those > to me somehow (the Data, Index, and Filter files) along with the > ColumnFamily definition from your config so I can try some tests out? > > -Jonathan > > On Thu, Oct 15, 2009 at 12:51 PM, Edmond Lau wrote: >> I'm using the cassandra 0.4 release. I was loading a bunch of data >> into cassandra when the thrift api started throwing >> UnavailableExceptions. Checking the logs, I found errors that looked >> like the following: >> >> ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 >> DebuggableThreadPoolExecutor.java (line 85) Error in >> ThreadPoolExecutor >> java.lang.IllegalArgumentException: The name should match the name of >> the current column or super column >> at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >> at >> org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >> at >> org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >> at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) >> at org.apache.cassandra.db.Memtable.put(Memtable.java:139) >> at >> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) >> at org.apache.cassandra.db.Table.apply(Table.java:608) >> at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) >> at >> org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) >> at >> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >> at java.lang.Thread.run(Thread.java:619) >> ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 >> CassandraDaemon.java (line 71) Fatal exception in thread >> Thread[ROW-MUTATION-STAGE:2935,5,main] >> java.lang.IllegalArgumentException: The name should match the name of >> the current column or super column >> at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >> at >> org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >> at >> org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >> at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) >> at org.apache.cassandra.db.Memtable.put(Memtable.java:139) >> at >> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) >> at org.apache.cassandra.db.Table.apply(Table.java:608) >> at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) >> at >> org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) >> at >> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >> at java.lang.Thread.run(Thread.java:619) >> >> Stopping and starting the cluster gives me something similar: >> >> ERROR - Error in executor futuretask >> java.util.concurrent.ExecutionException: >> java.lang.IllegalArgumentException: The name should match the name of >> the current column or super column >> at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) >> at java.util.concurrent.FutureTask.get(FutureTask.java:83) >> at >> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) >> at >> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >> at java.lang.Thread.run(Thread.java:619) >> Caused by: java.lang.IllegalArgumentException: The name should match >> the name of the current column or super column >> at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >> at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >> at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >> at org.apache.cassandra.db.ColumnFamily.resolve(ColumnFamily.java:408) >> at >> org.apache.cassandra.db.ColumnFamilyStore.merge(ColumnFamilyStore.java:477) >> at >> org.apache.cassandra.db.ColumnFamilyStore.doFileCompaction(ColumnFamilyStore.java:1078) >> at >> org.apache.cassandra.db.ColumnFamily
Re: cassandra fatal error - "The name should match the name of the current column or super column"
Yes, I'm using UTF8Type as the comparator. On Thu, Oct 15, 2009 at 11:33 AM, Jonathan Ellis wrote: > Quicker question: Is this ColumnFamily using UTF8Type as its comparator? > > On Thu, Oct 15, 2009 at 1:17 PM, Jonathan Ellis wrote: >> The exception on restart is occurring during a compaction of >> already-written sstables. It logs what the files are beforehand >> ("Compacting [...]"). Could you gzip up those sstables and send those >> to me somehow (the Data, Index, and Filter files) along with the >> ColumnFamily definition from your config so I can try some tests out? >> >> -Jonathan >> >> On Thu, Oct 15, 2009 at 12:51 PM, Edmond Lau wrote: >>> I'm using the cassandra 0.4 release. I was loading a bunch of data >>> into cassandra when the thrift api started throwing >>> UnavailableExceptions. Checking the logs, I found errors that looked >>> like the following: >>> >>> ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 >>> DebuggableThreadPoolExecutor.java (line 85) Error in >>> ThreadPoolExecutor >>> java.lang.IllegalArgumentException: The name should match the name of >>> the current column or super column >>> at >>> org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >>> at >>> org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >>> at >>> org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >>> at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) >>> at org.apache.cassandra.db.Memtable.put(Memtable.java:139) >>> at >>> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) >>> at org.apache.cassandra.db.Table.apply(Table.java:608) >>> at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) >>> at >>> org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) >>> at >>> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >>> at java.lang.Thread.run(Thread.java:619) >>> ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 >>> CassandraDaemon.java (line 71) Fatal exception in thread >>> Thread[ROW-MUTATION-STAGE:2935,5,main] >>> java.lang.IllegalArgumentException: The name should match the name of >>> the current column or super column >>> at >>> org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >>> at >>> org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >>> at >>> org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >>> at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) >>> at org.apache.cassandra.db.Memtable.put(Memtable.java:139) >>> at >>> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) >>> at org.apache.cassandra.db.Table.apply(Table.java:608) >>> at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) >>> at >>> org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) >>> at >>> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >>> at java.lang.Thread.run(Thread.java:619) >>> >>> Stopping and starting the cluster gives me something similar: >>> >>> ERROR - Error in executor futuretask >>> java.util.concurrent.ExecutionException: >>> java.lang.IllegalArgumentException: The name should match the name of >>> the current column or super column >>> at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) >>> at java.util.concurrent.FutureTask.get(FutureTask.java:83) >>> at >>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) >>> at >>> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) >>> at >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) >>> at java.lang.Thread.run(Thread.java:619) >>> Caused by: java.lang.IllegalArgumentException: The name should match >>> the name of the current column or super column >>> at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) >>> at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) >>> at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) >>> at org.apache.cassandra.db.ColumnFamily.resolve(ColumnFamily.java:408) >>> at >>> org.apache.cas
Re: Thrift Perl API Timeout Issues
I see a similar thing happening all the time. I get around it by closing the current connection and reconnecting after a sleep. Although I am able to do quite a few inserts between errors, so I'm not sure if it's the exact problem. -Anthony On Thu, Oct 15, 2009 at 11:26:08AM -0400, Eric Lubow wrote: > Using the Thrift Perl API into Cassandra, I am running into what is > endearingly referred to as the 4 bytes of doom: > TSocket: timed out reading 4 bytes from localhost:9160 > > The script I am using is fairly simple. I have a text file that has about > 3.6 million lines that are formatted like: f...@bar.com 1234 > > The Cassandra dataset is a single column family called Users in the Mailings > keyspace with a data layout of: > Users = { > 'f...@example.com': { > email: 'f...@example.com', > person_id: '123456', > send_dates_2009-09-30: '2245', > send_dates_2009-10-01: '2247', > }, > } > There are about 3.5 million rows in the Users column family and each row has > no more than 4 columns (listed above). Some only have 3 (one of the > send_dates_-MM-DD isn't there). > > The script parses it and then connects to Cassandra and does a get_slice and > counts the return values adding that to a hash: > my ($value) = $client->get_slice( > 'Mailings', > $email, > Cassandra::ColumnParent->new({ > column_family => 'Users', > }), > Cassandra::SlicePredicate->new({ > slice_range => Cassandra::SliceRange->new({ > start => 'send_dates_2009-09-29', > finish => 'send_dates_2009-10-30', > }), > }), > Cassandra::ConsistencyLevel::ONE > ); > $counter{($#{$value} + 1)}++; > > For the most part, this script times out after 1 minute or so. Replacing the > get_slice with a get_count, I can get it to about 2 million queries before I > get the timeout. Replacing the get_slice with a get, I make it to about 2.5 > million before I get the timeout. The only way I could get it to run all > the way through was to add a 1/100 of a second sleep during every iteration. > I was able to get the script to complete when I shut down everything else > on the machine (and it took 177m to complete). But since this is a > semi-production machine, I had to turn everything back on afterwards. > > So for poops and laughs (at the recommendation of jbellis), I rewrote the > script in Python and it has since run (using get_slice) 3 times fully > without timing out (approximately 130m in Python) with everything else > running on the machine. > > My question is, having seen this same thing in the PHP API and it is my > understanding that the Perl API was based on the PHP API, could > http://issues.apache.org/jira/browse/THRIFT-347 apply to Perl here too? Is > anyone else seeing this issue? If so, have you gotten around it? > > Thanks. > > -e -- Anthony Molinaro
Re: cassandra fatal error - "The name should match the name of the current column or super column"
Aha! :) Could you test the patch attached to https://issues.apache.org/jira/browse/CASSANDRA-493 ? (If you're using the binary release, you can get the source from https://svn.apache.org/repos/asf/incubator/cassandra/tags/cassandra-0.4.0-final/ and build with "ant") thanks, -Jonathan On Thu, Oct 15, 2009 at 1:35 PM, Edmond Lau wrote: > Yes, I'm using UTF8Type as the comparator. > > On Thu, Oct 15, 2009 at 11:33 AM, Jonathan Ellis wrote: >> Quicker question: Is this ColumnFamily using UTF8Type as its comparator? >> >> On Thu, Oct 15, 2009 at 1:17 PM, Jonathan Ellis wrote: >>> The exception on restart is occurring during a compaction of >>> already-written sstables. It logs what the files are beforehand >>> ("Compacting [...]"). Could you gzip up those sstables and send those >>> to me somehow (the Data, Index, and Filter files) along with the >>> ColumnFamily definition from your config so I can try some tests out? >>> >>> -Jonathan >>> >>> On Thu, Oct 15, 2009 at 12:51 PM, Edmond Lau wrote: I'm using the cassandra 0.4 release. I was loading a bunch of data into cassandra when the thrift api started throwing UnavailableExceptions. Checking the logs, I found errors that looked like the following: ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 DebuggableThreadPoolExecutor.java (line 85) Error in ThreadPoolExecutor java.lang.IllegalArgumentException: The name should match the name of the current column or super column at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) at org.apache.cassandra.db.Memtable.put(Memtable.java:139) at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) at org.apache.cassandra.db.Table.apply(Table.java:608) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 CassandraDaemon.java (line 71) Fatal exception in thread Thread[ROW-MUTATION-STAGE:2935,5,main] java.lang.IllegalArgumentException: The name should match the name of the current column or super column at org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) at org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) at org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) at org.apache.cassandra.db.Memtable.put(Memtable.java:139) at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) at org.apache.cassandra.db.Table.apply(Table.java:608) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) Stopping and starting the cluster gives me something similar: ERROR - Error in executor futuretask java.util.concurrent.ExecutionException: java.lang.IllegalArgumentException: The name should match the name of the current column or super column at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:888) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:619) Caused by: java.
Re: Thrift Perl API Timeout Issues
Are you also using Perl? On Thu, Oct 15, 2009 at 1:38 PM, Anthony Molinaro wrote: > I see a similar thing happening all the time. I get around it by closing > the current connection and reconnecting after a sleep. Although I am able > to do quite a few inserts between errors, so I'm not sure if it's the > exact problem. > > -Anthony
Re: cassandra fatal error - "The name should match the name of the current column or super column"
Thanks Jonathan for the fast response time. I'll patch your fix and give it a whirl. On Thu, Oct 15, 2009 at 11:47 AM, Jonathan Ellis wrote: > Aha! :) > > Could you test the patch attached to > https://issues.apache.org/jira/browse/CASSANDRA-493 ? > > (If you're using the binary release, you can get the source from > https://svn.apache.org/repos/asf/incubator/cassandra/tags/cassandra-0.4.0-final/ > and build with "ant") > > thanks, > > -Jonathan > > On Thu, Oct 15, 2009 at 1:35 PM, Edmond Lau wrote: >> Yes, I'm using UTF8Type as the comparator. >> >> On Thu, Oct 15, 2009 at 11:33 AM, Jonathan Ellis wrote: >>> Quicker question: Is this ColumnFamily using UTF8Type as its comparator? >>> >>> On Thu, Oct 15, 2009 at 1:17 PM, Jonathan Ellis wrote: The exception on restart is occurring during a compaction of already-written sstables. It logs what the files are beforehand ("Compacting [...]"). Could you gzip up those sstables and send those to me somehow (the Data, Index, and Filter files) along with the ColumnFamily definition from your config so I can try some tests out? -Jonathan On Thu, Oct 15, 2009 at 12:51 PM, Edmond Lau wrote: > I'm using the cassandra 0.4 release. I was loading a bunch of data > into cassandra when the thrift api started throwing > UnavailableExceptions. Checking the logs, I found errors that looked > like the following: > > ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,518 > DebuggableThreadPoolExecutor.java (line 85) Error in > ThreadPoolExecutor > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at > org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) > at > org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) > at > org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) > at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) > at org.apache.cassandra.db.Memtable.put(Memtable.java:139) > at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) > at org.apache.cassandra.db.Table.apply(Table.java:608) > at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) > at > org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > ERROR [ROW-MUTATION-STAGE:2935] 2009-10-15 17:32:52,519 > CassandraDaemon.java (line 71) Fatal exception in thread > Thread[ROW-MUTATION-STAGE:2935,5,main] > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at > org.apache.cassandra.db.SuperColumn.putColumn(SuperColumn.java:208) > at > org.apache.cassandra.db.ColumnFamily.addColumn(ColumnFamily.java:200) > at > org.apache.cassandra.db.ColumnFamily.addColumns(ColumnFamily.java:127) > at org.apache.cassandra.db.Memtable.resolve(Memtable.java:156) > at org.apache.cassandra.db.Memtable.put(Memtable.java:139) > at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:450) > at org.apache.cassandra.db.Table.apply(Table.java:608) > at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:205) > at > org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:79) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:39) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > > Stopping and starting the cluster gives me something similar: > > ERROR - Error in executor futuretask > java.util.concurrent.ExecutionException: > java.lang.IllegalArgumentException: The name should match the name of > the current column or super column > at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) > at java.util.concurrent.FutureTask.get(FutureTask.java:83) > at > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logFutureExceptions(DebuggableThreadPoolExecutor.java:95) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor.afterExecute(DebuggableScheduledThreadPoolExecutor.java:50) > a
Re: Thrift Perl API Timeout Issues
So I ran the tests again twice with a huge timeout and it managed to run in just under 3 hours both times. So this issue is definitely related to the timeouts. It might be worth changing the default timeouts for Perl to match the infinite timeouts for Python. Thanks for the quick responses. -e On Thu, Oct 15, 2009 at 2:48 PM, Jonathan Ellis wrote: > Are you also using Perl? > > On Thu, Oct 15, 2009 at 1:38 PM, Anthony Molinaro > wrote: > > I see a similar thing happening all the time. I get around it by closing > > the current connection and reconnecting after a sleep. Although I am > able > > to do quite a few inserts between errors, so I'm not sure if it's the > > exact problem. > > > > -Anthony >