You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Teodor Sigaev <te...@sigaev.ru> on 2009/10/14 15:17:08 UTC

Cassandra restart

I have 12-nodes cassandra's cluster, after restart (really, upgrade from 0.4.0 
to 0.4.1) of one node it writes in log a lot of messages like:
ERROR [pool-1-thread-46] 2009-10-14 17:06:40,804 Cassandra.java (line 679) 
Internal error processing get_slice
java.lang.RuntimeException: error reading key 00000000000028344851
         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:100)
         at 
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
         at 
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
         at 
org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
         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


Read from that key from another nodes is ok, but reading from restarted 
timeouted with that message.

Is some rules/recommendation to restart node?



-- 
Teodor Sigaev                                   E-mail: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
> 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: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Jonathan Ellis <jb...@gmail.com>.
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 <te...@sigaev.ru>:
>> 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='0000000000000000000000000000000000849706',
> column_parent='QueryPath(columnFamilyName='Super1',
> superColumnName='[B@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='0000000000000000000000000000000000849706',
> column_parent='QueryPath(columnFamilyName='Super1',
> superColumnName='[B@6ca50fbe', columnName='null')', start='1', finish='0',
> reversed=true, count=2) from 207911@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
> 0000000000000000000000000000000000849706
>    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 0000000000000000000000000000000000849706; sending response to
> 207911@10.3.2.38:7000
> ....
> DEBUG [CONSISTENCY-MANAGER:2] 2009-10-15 14:18:16,308
> ConsistencyManager.java (line 168) Reading consistency digest for
> 0000000000000000000000000000000000849706 from 527021@[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:
>  <ReplicationFactor>3</ReplicationFactor>
>  <FlushMinThreads>8</FlushMinThreads>
>  <FlushMaxThreads>16</FlushMaxThreads>
> - 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: teodor@sigaev.ru
>                                                   WWW: http://www.sigaev.ru/
>

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
> 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='0000000000000000000000000000000000849706', 
column_parent='QueryPath(columnFamilyName='Super1', 
superColumnName='[B@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='0000000000000000000000000000000000849706', 
column_parent='QueryPath(columnFamilyName='Super1', 
superColumnName='[B@6ca50fbe', columnName='null')', start='1', finish='0', 
reversed=true, count=2) from 207911@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 
0000000000000000000000000000000000849706
     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 0000000000000000000000000000000000849706; sending response to 
207911@10.3.2.38:7000
....
DEBUG [CONSISTENCY-MANAGER:2] 2009-10-15 14:18:16,308 ConsistencyManager.java 
(line 168) Reading consistency digest for 
0000000000000000000000000000000000849706 from 527021@[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:
   <ReplicationFactor>3</ReplicationFactor>
   <FlushMinThreads>8</FlushMinThreads>
   <FlushMaxThreads>16</FlushMaxThreads>
- 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: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
Sorry, I don't see anything connected to key 00000000000013062900 or message 
2691, debug is turned off on that node because 10.3.2.36 wasn't restarted

I will try to reproduce problem on smaller test cluster.

Fragment of 10.3.2.36 log (time is in sync):
ERROR [pool-1-thread-15051] 2009-10-14 18:03:30,562 StorageProxy.java (line 179) 
error writing key 00000000000176715312
java.util.concurrent.TimeoutException: Operation timed out - received only 0 
responses from  .
     at 
org.apache.cassandra.service.QuorumResponseHandler.get(QuorumResponseHandler.java:88)
     at 
org.apache.cassandra.service.StorageProxy.insertBlocking(StorageProxy.java:164)
     at 
org.apache.cassandra.service.CassandraServer.doInsert(CassandraServer.java:468)
     at 
org.apache.cassandra.service.CassandraServer.remove(CassandraServer.java:461)
     at 
org.apache.cassandra.service.Cassandra$Processor$remove.process(Cassandra.java:884)
     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)
ERROR [pool-1-thread-14947] 2009-10-14 18:03:32,593 StorageProxy.java (line 179) 
error writing key 00000000000176715312
java.util.concurrent.TimeoutException: Operation timed out - received only 0 
responses from  .
     at 
org.apache.cassandra.service.QuorumResponseHandler.get(QuorumResponseHandler.java:88)
     at 
org.apache.cassandra.service.StorageProxy.insertBlocking(StorageProxy.java:164)
     at 
org.apache.cassandra.service.CassandraServer.doInsert(CassandraServer.java:468)
     at 
org.apache.cassandra.service.CassandraServer.remove(CassandraServer.java:461)
     at 
org.apache.cassandra.service.Cassandra$Processor$remove.process(Cassandra.java:884)
     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)
ERROR [pool-1-thread-14985] 2009-10-14 18:03:34,571 StorageProxy.java (line 179) 
error writing key 00000000000176715312
java.util.concurrent.TimeoutException: Operation timed out - received only 0 
responses from  .
     at 
org.apache.cassandra.service.QuorumResponseHandler.get(QuorumResponseHandler.java:88)
     at 
org.apache.cassandra.service.StorageProxy.insertBlocking(StorageProxy.java:164)
     at 
org.apache.cassandra.service.CassandraServer.doInsert(CassandraServer.java:468)
     at 
org.apache.cassandra.service.CassandraServer.remove(CassandraServer.java:461)
     at 
org.apache.cassandra.service.Cassandra$Processor$remove.process(Cassandra.java:884)
     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)
ERROR [pool-1-thread-14493] 2009-10-14 18:03:34,571 Cassandra.java (line 679) 
Internal error processing get_slice
java.lang.RuntimeException: error reading key 00000000000106371503
     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:100)
     at 
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
     at 
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
     at 
org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
     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
ERROR [pool-1-thread-14820] 2009-10-14 18:03:35,406 Cassandra.java (line 679) 
Internal error processing get_slice
java.lang.RuntimeException: error reading key 00000000000019586353
     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:100)
     at 
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
     at 
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
     at 
org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
     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
ERROR [pool-1-thread-15051] 2009-10-14 18:03:36,602 StorageProxy.java (line 179) 
error writing key 00000000000176715312
java.util.concurrent.TimeoutException: Operation timed out - received only 0 
responses from  .
     at 
org.apache.cassandra.service.QuorumResponseHandler.get(QuorumResponseHandler.java:88)
     at 
org.apache.cassandra.service.StorageProxy.insertBlocking(StorageProxy.java:164)
     at 
org.apache.cassandra.service.CassandraServer.doInsert(CassandraServer.java:468)
     at 
org.apache.cassandra.service.CassandraServer.remove(CassandraServer.java:461)
     at 
org.apache.cassandra.service.Cassandra$Processor$remove.process(Cassandra.java:884)
     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)
ERROR [pool-1-thread-14658] 2009-10-14 18:03:37,590 Cassandra.java (line 679) 
Internal error processing get_slice
java.lang.RuntimeException: error reading key 00000000000139616600
     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:100)
     at 
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
     at 
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
     at 
org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
     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


-- 
Teodor Sigaev                                   E-mail: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Jonathan Ellis <jb...@gmail.com>.
And what happens on 10.3.2.36 for message 2691?

2009/10/14 Teodor Sigaev <te...@sigaev.ru>:
> After restart old key become readable, another key become unreadable :)
>
> DEBUG [pool-1-thread-61] 2009-10-14 18:03:30,695 StorageProxy.java (line
> 239) weakreadlocal reading SliceFromReadCommand(table='XXXX',
> key='00000000000013062900',
> column_parent='QueryPath(columnFamilyName='UserMapToX',
> superColumnName='[B@7edea8b3', columnName='null')',
> start='99999999999999999999:', finish='-00000000000000000000:',
> reversed=true, count=2)
> DEBUG [pool-1-thread-61] 2009-10-14 18:03:30,695 StorageProxy.java (line
> 251) weakreadremote reading SliceFromReadCommand(table='XXXX',
> key='00000000000013062900',
> column_parent='QueryPath(columnFamilyName='UserMapToX',
> superColumnName='[B@7edea8b3', columnName='null')',
> start='99999999999999999999:', finish='-00000000000000000000:',
> reversed=true, count=2) from 2691@10.3.2.36:7000
> ...
> ERROR [pool-1-thread-61] 2009-10-14 18:03:35,688 Cassandra.java (line 679)
> Internal error processing get_slice
> java.lang.RuntimeException: error reading key 00000000000013062900
>        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:100)
>        at
> org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
>        at
> org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
>        at
> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
>        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
> DEBUG [pool-1-thread-61] 2009-10-14 18:03:35,740 CassandraServer.java (line
> 455) remove
>
>
>
>
> --
> Teodor Sigaev                                   E-mail: teodor@sigaev.ru
>                                                   WWW: http://www.sigaev.ru/
>

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
After restart old key become readable, another key become unreadable :)

DEBUG [pool-1-thread-61] 2009-10-14 18:03:30,695 StorageProxy.java (line 239) 
weakreadlocal reading SliceFromReadCommand(table='XXXX', 
key='00000000000013062900', 
column_parent='QueryPath(columnFamilyName='UserMapToX', 
superColumnName='[B@7edea8b3', columnName='null')', 
start='99999999999999999999:', finish='-00000000000000000000:', reversed=true, 
count=2)
DEBUG [pool-1-thread-61] 2009-10-14 18:03:30,695 StorageProxy.java (line 251) 
weakreadremote reading SliceFromReadCommand(table='XXXX', 
key='00000000000013062900', 
column_parent='QueryPath(columnFamilyName='UserMapToX', 
superColumnName='[B@7edea8b3', columnName='null')', 
start='99999999999999999999:', finish='-00000000000000000000:', reversed=true, 
count=2) from 2691@10.3.2.36:7000
...
ERROR [pool-1-thread-61] 2009-10-14 18:03:35,688 Cassandra.java (line 679) 
Internal error processing get_slice
java.lang.RuntimeException: error reading key 00000000000013062900
         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:100)
         at 
org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
         at 
org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
         at 
org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
         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
DEBUG [pool-1-thread-61] 2009-10-14 18:03:35,740 CassandraServer.java (line 455) 
remove




-- 
Teodor Sigaev                                   E-mail: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Jonathan Ellis <jb...@gmail.com>.
Turn on debug logging and see what nodes it is sending requests to,
and what happens on that node to that request.

Maybe there is an exception on the data node.

2009/10/14 Teodor Sigaev <te...@sigaev.ru>:
>> Why is it timing out?
>
> Sorry, I can't say why, I just see on client side
> {
>                 'code' => 0,
>                 'message' => 'TSocket: timed out reading 4 bytes from
> 10.3.2.25:9160'
>               }, 'Thrift::TException'
>
> and it's repeatable for this key, for another key it could be ok.
> And on server side:
>
>>> ERROR [pool-1-thread-46] 2009-10-14 17:06:40,804 Cassandra.java (line
>>> 679)
>>> Internal error processing get_slice
>>> java.lang.RuntimeException: error reading key 00000000000028344851
>>>       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:100)
>>>       at
>>>
>>> org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
>>>       at
>>>
>>> org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
>>>       at
>>>
>>> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
>>>       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
>
> --
> Teodor Sigaev                                   E-mail: teodor@sigaev.ru
>                                                   WWW: http://www.sigaev.ru/
>

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
Additional info. Cluster mostly in write mode, it have about 2000-5000 
insert/delete requests per second and only 10-100 read requests per second.



-- 
Teodor Sigaev                                   E-mail: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Teodor Sigaev <te...@sigaev.ru>.
> Why is it timing out?
Sorry, I can't say why, I just see on client side
{
                  'code' => 0,
                  'message' => 'TSocket: timed out reading 4 bytes from 
10.3.2.25:9160'
                }, 'Thrift::TException'

and it's repeatable for this key, for another key it could be ok.
And on server side:

>> ERROR [pool-1-thread-46] 2009-10-14 17:06:40,804 Cassandra.java (line 679)
>> Internal error processing get_slice
>> java.lang.RuntimeException: error reading key 00000000000028344851
>>        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:100)
>>        at
>> org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
>>        at
>> org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
>>        at
>> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
>>        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

-- 
Teodor Sigaev                                   E-mail: teodor@sigaev.ru
                                                    WWW: http://www.sigaev.ru/

Re: Cassandra restart

Posted by Jonathan Ellis <jb...@gmail.com>.
Why is it timing out?

2009/10/14 Teodor Sigaev <te...@sigaev.ru>:
> I have 12-nodes cassandra's cluster, after restart (really, upgrade from
> 0.4.0 to 0.4.1) of one node it writes in log a lot of messages like:
> ERROR [pool-1-thread-46] 2009-10-14 17:06:40,804 Cassandra.java (line 679)
> Internal error processing get_slice
> java.lang.RuntimeException: error reading key 00000000000028344851
>        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:100)
>        at
> org.apache.cassandra.service.CassandraServer.getSlice(CassandraServer.java:182)
>        at
> org.apache.cassandra.service.CassandraServer.multigetSliceInternal(CassandraServer.java:251)
>        at
> org.apache.cassandra.service.CassandraServer.get_slice(CassandraServer.java:220)
>        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
>
>
> Read from that key from another nodes is ok, but reading from restarted
> timeouted with that message.
>
> Is some rules/recommendation to restart node?
>
>
>
> --
> Teodor Sigaev                                   E-mail: teodor@sigaev.ru
>                                                   WWW: http://www.sigaev.ru/
>