[ 
https://issues.apache.org/jira/browse/CASSANDRA-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13163721#comment-13163721
 ] 

Dominic Williams commented on CASSANDRA-3551:
---------------------------------------------

I hit a version of this problem...

I upgraded a production cluster from 1.0.3 (from a non-official version patched 
for CASSANDRA-3510) to 1.0.5. The aim was to pass CASSANDRA-3440.

This generated a timeout storm on range slices and I have reverted. 

Notes:

1/ The 1.0.5 node CPUs all showed tiny load - in fact, they seemed to be 
substantially less loaded than the 1.0.3 nodes were/are again

2/ The system.log files on the 1.0.5 nodes didn't record any errors

3/ range_slice timeout storm experienced in application layer. Example log 
trace below

org.apache.thrift.transport.TTransportException: 
java.net.SocketTimeoutException: Read timed out
        at 
org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129)
 ~[libthrift-0.6.1.jar:0.6.1]
        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84) 
~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.thrift.transport.TFramedTransport.readFrame(TFramedTransport.java:129)
 ~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.thrift.transport.TFramedTransport.read(TFramedTransport.java:101) 
~[libthrift-0.6.1.jar:0.6.1]
        at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84) 
~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:378) 
~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:297) 
~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:204)
 ~[libthrift-0.6.1.jar:0.6.1]
        at 
org.apache.cassandra.thrift.Cassandra$Client.recv_get_slice(Cassandra.java:560) 
~[cassandra-thrift-1.0.1.jar:1.0.1]
        at 
org.apache.cassandra.thrift.Cassandra$Client.get_slice(Cassandra.java:542) 
~[cassandra-thrift-1.0.1.jar:1.0.1]
        at org.scale7.cassandra.pelops.Selector$3.execute(Selector.java:683) 
~[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at org.scale7.cassandra.pelops.Selector$3.execute(Selector.java:680) 
~[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at org.scale7.cassandra.pelops.Operand.tryOperation(Operand.java:86) 
[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at org.scale7.cassandra.pelops.Operand.tryOperation(Operand.java:66) 
[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at 
org.scale7.cassandra.pelops.Selector.getColumnOrSuperColumnsFromRow(Selector.java:680)
 [scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at 
org.scale7.cassandra.pelops.Selector.getColumnsFromRow(Selector.java:689) 
[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at 
org.scale7.cassandra.pelops.Selector.getColumnsFromRow(Selector.java:676) 
[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at 
org.scale7.cassandra.pelops.Selector.getColumnsFromRow(Selector.java:562) 
[scale7-pelops-1.3-1.0.x-SNAPSHOT.jar:na]
        at com.fightmymonster.game.Monsters.getMonster(Monsters.java:92) 
[fmmServer.jar:na]
        at 
com.fightmymonster.rmi.monsters.GetMonster.doWork(GetMonster.java:25) 
[fmmServer.jar:na]
        at 
org.wyki.networking.starburst.SyncRmiOperation.run(SyncRmiOperation.java:50) 
[fmmServer.jar:na]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
 [na:1.6.0_22]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) 
[na:1.6.0_22]
        at java.lang.Thread.run(Thread.java:662) [na:1.6.0_22]
Caused by: java.net.SocketTimeoutException: Read timed out
        at java.net.SocketInputStream.socketRead0(Native Method) ~[na:1.6.0_22]
        at java.net.SocketInputStream.read(SocketInputStream.java:129) 
~[na:1.6.0_22]
        at 
org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
 ~[libthrift-0.6.1.jar:0.6.1]
        ... 23 common frames omitted
                
> Timeout exception for Range Slice after upgrade from 1.0.2 to 1.0.5 
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-3551
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3551
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.0.5
>         Environment: Linux, Cassandra 1.0.5
>            Reporter: Zhong Li
>            Priority: Critical
>
> I upgraded from 1.0.2 to 1.0.5. For some column families always got 
> TimeoutException. I turned on debug and increase rpc_timeout to 1 minute, but 
> still got timeout. I believe it is bug on 1.0.5.
> ConsistencyLevel is QUORUM, replicate factor is 3. 
> Here are partial logs. 
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,717 StorageProxy.java (line 813) 
> RangeSliceCommand{keyspace='keyspaceLBSDATAPRODUS', 
> column_family='dataProvider', super_column=null, predicate=SlicePre
> dicate(slice_range:SliceRange(start:80 01 00 01 00 00 00 10 67 65 74 5F 72 61 
> 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 00 0C 64 61 
> 74 61 50 72 6F 76 69 64 65 72 00 0C 00 0
> 2 0C 00 02 0B 00 01 00 00 00 00, finish:80 01 00 01 00 00 00 10 67 65 74 5F 
> 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 00 0C 
> 64 61 74 61 50 72 6F 76 69 64 65 72 00 0C 
> 00 02 0C 00 02 0B 00 01 00 00 00 00 0B 00 02 00 00 00 00, reversed:false, 
> count:1024)), range=[PROD/US/000/0,PROD/US/999/99999], max_keys=1024}
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,718 StorageProxy.java (line 1012) 
> restricted ranges for query [PROD/US/000/0,PROD/US/999/99999] are 
> [[PROD/US/000/0,PROD/US/300/~], (PROD/US/300/~,PROD/
> US/600/~], (PROD/US/600/~,PROD/US/999/99999]]
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,720 VoxeoStrategy.java (line 157) 
> ReplicationFactor 3
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,720 VoxeoStrategy.java (line 33) 
> PROD/US/300/~
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,721 VoxeoStrategy.java (line 96) 
> End region for token PROD/US/300/~ PROD/US/300/~ 10.92.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,721 VoxeoStrategy.java (line 96) 
> End region for token PROD/US/300/~ PROD/US/600/~ 10.72.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,721 VoxeoStrategy.java (line 96) 
> End region for token PROD/US/300/~ PROD/US/999/~ 10.8.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,723 VoxeoStrategy.java (line 157) 
> ReplicationFactor 3
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,724 ReadCallback.java (line 77) 
> Blockfor/repair is 2/false; setting up requests to 
> /10.92.208.103,/10.72.208.103
> DEBUG [WRITE-/10.92.208.103] 2011-12-01 22:25:39,725 
> OutboundTcpConnection.java (line 206) attempting to connect to /10.92.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,726 StorageProxy.java (line 859) 
> reading RangeSliceCommand{keyspace='keyspaceLBSDATAPRODUS', 
> column_family='dataProvider', super_column=null, predicate=
> SlicePredicate(slice_range:SliceRange(start:80 01 00 01 00 00 00 10 67 65 74 
> 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 00 
> 0C 64 61 74 61 50 72 6F 76 69 64 65 72 00
>  0C 00 02 0C 00 02 0B 00 01 00 00 00 00, finish:80 01 00 01 00 00 00 10 67 65 
> 74 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 
> 00 0C 64 61 74 61 50 72 6F 76 69 64 65 7
> 2 00 0C 00 02 0C 00 02 0B 00 01 00 00 00 00 0B 00 02 00 00 00 00, 
> reversed:false, count:1024)), range=[PROD/US/000/0,PROD/US/300/~], 
> max_keys=1024} from /10.92.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,726 StorageProxy.java (line 859) 
> reading RangeSliceCommand{keyspace='keyspaceLBSDATAPRODUS', 
> column_family='dataProvider', super_column=null, predicate=
> SlicePredicate(slice_range:SliceRange(start:80 01 00 01 00 00 00 10 67 65 74 
> 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 00 
> 0C 64 61 74 61 50 72 6F 76 69 64 65 72 00
>  0C 00 02 0C 00 02 0B 00 01 00 00 00 00, finish:80 01 00 01 00 00 00 10 67 65 
> 74 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 
> 00 0C 64 61 74 61 50 72 6F 76 69 64 65 7
> 2 00 0C 00 02 0C 00 02 0B 00 01 00 00 00 00 0B 00 02 00 00 00 00, 
> reversed:false, count:1024)), range=[PROD/US/000/0,PROD/US/300/~], 
> max_keys=1024} from /10.72.208.103
> DEBUG [WRITE-/10.8.208.103] 2011-12-01 22:25:39,727 
> OutboundTcpConnection.java (line 206) attempting to connect to /10.8.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,727 StorageProxy.java (line 859) 
> reading RangeSliceCommand{keyspace='keyspaceLBSDATAPRODUS', 
> column_family='dataProvider', super_column=null, predicate=
> SlicePredicate(slice_range:SliceRange(start:80 01 00 01 00 00 00 10 67 65 74 
> 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 00 
> 0C 64 61 74 61 50 72 6F 76 69 64 65 72 00
>  0C 00 02 0C 00 02 0B 00 01 00 00 00 00, finish:80 01 00 01 00 00 00 10 67 65 
> 74 5F 72 61 6E 67 65 5F 73 6C 69 63 65 73 00 00 00 03 0C 00 01 0B 00 03 00 00 
> 00 0C 64 61 74 61 50 72 6F 76 69 64 65 7
> 2 00 0C 00 02 0C 00 02 0B 00 01 00 00 00 00 0B 00 02 00 00 00 00, 
> reversed:false, count:1024)), range=[PROD/US/000/0,PROD/US/300/~], 
> max_keys=1024} from /10.8.208.103
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,731 SliceQueryFilter.java (line 123) 
> collecting 0 of 1024: active:false:1@1322777621601000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,731 SliceQueryFilter.java (line 123) 
> collecting 1 of 1024: name:false:4@1322777621601000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,731 SliceQueryFilter.java (line 123) 
> collecting 2 of 1024: providerData:false:2283@1321549067179000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,731 SliceQueryFilter.java (line 123) 
> collecting 3 of 1024: providerID:false:1@1322777621601000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,732 SliceQueryFilter.java (line 123) 
> collecting 4 of 1024: timestamp:false:13@1322777621601000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,732 SliceQueryFilter.java (line 123) 
> collecting 5 of 1024: vendorData:false:2364@1322777621601000
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,733 ColumnFamilyStore.java (line 
> 1331) scanned DecoratedKey(PROD/US/001/1, 50524f442f55532f3030312f31)
> DEBUG [ReadStage:1] 2011-12-01 22:25:39,733 RangeSliceVerbHandler.java (line 
> 55) Sending RangeSliceReply{rows=Row(key=DecoratedKey(PROD/US/001/1, 
> 50524f442f55532f3030312f31), cf=ColumnFamily(dataP
> rovider 
> [active:false:1@1322777621601000,name:false:4@1322777621601000,providerData:false:2283@1321549067179000,providerID:false:1@1322777621601000,timestamp:false:13@1322777621601000,vendorData:f
> alse:2364@1322777621601000,]))} to 72@/10.72.208.103
> DEBUG [RequestResponseStage:1] 2011-12-01 22:25:39,734 
> ResponseVerbHandler.java (line 44) Processing response on a callback from 
> 72@/10.72.208.103
> DEBUG [RequestResponseStage:2] 2011-12-01 22:25:39,887 
> ResponseVerbHandler.java (line 44) Processing response on a callback from 
> 71@/10.92.208.103
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,889 SliceQueryFilter.java (line 
> 123) collecting 0 of 2147483647: active:false:1@1322777621601000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,890 SliceQueryFilter.java (line 
> 123) collecting 1 of 2147483647: name:false:4@1322777621601000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,890 SliceQueryFilter.java (line 
> 123) collecting 2 of 2147483647: providerData:false:2283@1321549067179000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,890 SliceQueryFilter.java (line 
> 123) collecting 3 of 2147483647: providerID:false:1@1322777621601000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,890 SliceQueryFilter.java (line 
> 123) collecting 4 of 2147483647: timestamp:false:13@1322777621601000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,891 SliceQueryFilter.java (line 
> 123) collecting 5 of 2147483647: vendorData:false:2364@1322777621601000
> DEBUG [pool-2-thread-1] 2011-12-01 22:25:39,892 StorageProxy.java (line 867) 
> range slices read DecoratedKey(PROD/US/001/1, 50524f442f55532f3030312f31)
> DEBUG [RequestResponseStage:3] 2011-12-01 22:25:39,936 
> ResponseVerbHandler.java (line 44) Processing response on a callback from 
> 73@/10.8.208.103
> DEBUG [ScheduledTasks:1] 2011-12-01 22:26:19,788 LoadBroadcaster.java (line 
> 86) Disseminating load info ...
> DEBUG [pool-2-thread-1] 2011-12-01 22:26:39,904 StorageProxy.java (line 874) 
> Range slice timeout: java.util.concurrent.TimeoutException: Operation timed 
> out.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to