Cassandra
  1. Cassandra
  2. CASSANDRA-3551

Timeout exception for quorum reads after upgrade from 1.0.2 to 1.0.5

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Critical Critical
    • Resolution: Fixed
    • Fix Version/s: 1.0.6
    • Component/s: Core
    • Labels:
      None
    • Environment:

      Linux, Cassandra 1.0.5

      Description

      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.

      1. 3551.patch
        1 kB
        Sylvain Lebresne

        Activity

        Zhong Li created issue -
        Hide
        Jonathan Ellis added a comment -

        any exceptions on the other nodes?

        Show
        Jonathan Ellis added a comment - any exceptions on the other nodes?
        Hide
        Janne Jalkanen added a comment -

        I am seeing this too; switching to ConsistencyLevel.ONE helps, but does not solve the problem completely, i.e. queries fail less often.

        Show
        Janne Jalkanen added a comment - I am seeing this too; switching to ConsistencyLevel.ONE helps, but does not solve the problem completely, i.e. queries fail less often.
        Hide
        Sylvain Lebresne added a comment - - edited

        More infos on you respective setups could help. For instance:

        • you said, 'For some column families'. Is there something specific to those column families ? Are they using compression? leveled compaction?
        • Janne: you're seeing it too, but on which version exactly did you definitively not see this problem and on which are you definitively seeing it? Is it 1.0.2 and 1.0.5 respectively as for Zhong?
        • As Jonathan said, are you seeing any error in any node logs?
        Show
        Sylvain Lebresne added a comment - - edited More infos on you respective setups could help. For instance: you said, 'For some column families'. Is there something specific to those column families ? Are they using compression? leveled compaction? Janne: you're seeing it too, but on which version exactly did you definitively not see this problem and on which are you definitively seeing it? Is it 1.0.2 and 1.0.5 respectively as for Zhong? As Jonathan said, are you seeing any error in any node logs?
        Hide
        Janne Jalkanen added a comment - - edited

        1.0.5, RF 3, 3 node cluster on EC2. I upgraded just recently directly from 0.6.13, so I have not been on any earlier 1.0.x version. No compression, just a straightforward upgrade with minimal tuning to the cassandra.yaml file. 2GB heap, maybe ~1GB in use. Happens with column families which have 20 rows, CFs which have 10000 rows and more. Happens when trying to read 100 rows at a time, happens when trying to read 10k rows at a time. The only factor that I've noticed while trying to tune that has any effect is changing the CL.

        No errors in node logs, no anomalies in system monitoring (like suddenly increased disk latency). Only cassandra's storageproxy latency goes way up (hundreds of milliseconds), before failure.

        Here is the exception from hector:

        Caused by: me.prettyprint.hector.api.exceptions.HTimedOutException: TimedOutException()
        at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:42)
        at me.prettyprint.cassandra.service.KeyspaceServiceImpl$3.execute(KeyspaceServiceImpl.java:163)
        at me.prettyprint.cassandra.service.KeyspaceServiceImpl$3.execute(KeyspaceServiceImpl.java:145)
        at me.prettyprint.cassandra.service.Operation.executeAndSetResult(Operation.java:101)
        at me.prettyprint.cassandra.connection.HConnectionManager.operateWithFailover(HConnectionManager.java:233)
        at me.prettyprint.cassandra.service.KeyspaceServiceImpl.operateWithFailover(KeyspaceServiceImpl.java:131)
        at me.prettyprint.cassandra.service.KeyspaceServiceImpl.getRangeSlices(KeyspaceServiceImpl.java:167)
        at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery$1.doInKeyspace(ThriftRangeSlicesQuery.java:67)
        at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery$1.doInKeyspace(ThriftRangeSlicesQuery.java:63)
        at me.prettyprint.cassandra.model.KeyspaceOperationCallback.doInKeyspaceAndMeasure(KeyspaceOperationCallback.java:20)
        at me.prettyprint.cassandra.model.ExecutingKeyspace.doExecute(ExecutingKeyspace.java:85)
        at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery.execute(ThriftRangeSlicesQuery.java:62)

        Here's the CF definition:

        ColumnFamily: XXXX
        Key Validation Class: org.apache.cassandra.db.marshal.BytesType
        Default column value validator: org.apache.cassandra.db.marshal.BytesType
        Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type
        Row cache size / save period in seconds / keys to save : 0.0/0/all
        Row Cache Provider: org.apache.cassandra.cache.ConcurrentLinkedHashCacheProvider
        Key cache size / save period in seconds: 200000.0/14400
        GC grace seconds: 864000
        Compaction min/max thresholds: 4/32
        Read repair chance: 1.0
        Replicate on write: true
        Built indexes: []
        Compaction Strategy: org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy

        Show
        Janne Jalkanen added a comment - - edited 1.0.5, RF 3, 3 node cluster on EC2. I upgraded just recently directly from 0.6.13, so I have not been on any earlier 1.0.x version. No compression, just a straightforward upgrade with minimal tuning to the cassandra.yaml file. 2GB heap, maybe ~1GB in use. Happens with column families which have 20 rows, CFs which have 10000 rows and more. Happens when trying to read 100 rows at a time, happens when trying to read 10k rows at a time. The only factor that I've noticed while trying to tune that has any effect is changing the CL. No errors in node logs, no anomalies in system monitoring (like suddenly increased disk latency). Only cassandra's storageproxy latency goes way up (hundreds of milliseconds), before failure. Here is the exception from hector: Caused by: me.prettyprint.hector.api.exceptions.HTimedOutException: TimedOutException() at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:42) at me.prettyprint.cassandra.service.KeyspaceServiceImpl$3.execute(KeyspaceServiceImpl.java:163) at me.prettyprint.cassandra.service.KeyspaceServiceImpl$3.execute(KeyspaceServiceImpl.java:145) at me.prettyprint.cassandra.service.Operation.executeAndSetResult(Operation.java:101) at me.prettyprint.cassandra.connection.HConnectionManager.operateWithFailover(HConnectionManager.java:233) at me.prettyprint.cassandra.service.KeyspaceServiceImpl.operateWithFailover(KeyspaceServiceImpl.java:131) at me.prettyprint.cassandra.service.KeyspaceServiceImpl.getRangeSlices(KeyspaceServiceImpl.java:167) at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery$1.doInKeyspace(ThriftRangeSlicesQuery.java:67) at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery$1.doInKeyspace(ThriftRangeSlicesQuery.java:63) at me.prettyprint.cassandra.model.KeyspaceOperationCallback.doInKeyspaceAndMeasure(KeyspaceOperationCallback.java:20) at me.prettyprint.cassandra.model.ExecutingKeyspace.doExecute(ExecutingKeyspace.java:85) at me.prettyprint.cassandra.model.thrift.ThriftRangeSlicesQuery.execute(ThriftRangeSlicesQuery.java:62) Here's the CF definition: ColumnFamily: XXXX Key Validation Class: org.apache.cassandra.db.marshal.BytesType Default column value validator: org.apache.cassandra.db.marshal.BytesType Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type Row cache size / save period in seconds / keys to save : 0.0/0/all Row Cache Provider: org.apache.cassandra.cache.ConcurrentLinkedHashCacheProvider Key cache size / save period in seconds: 200000.0/14400 GC grace seconds: 864000 Compaction min/max thresholds: 4/32 Read repair chance: 1.0 Replicate on write: true Built indexes: [] Compaction Strategy: org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy
        Hide
        Zhong Li added a comment -

        There is no exceptions on other nodes.

        I might be wrong about 'For some column families'. I saw another column family failed with Range Slice too. It works for insert. It might work for others retrieve command. I need test more when I have time. There is no compression, one row data only.

        ColumnFamily: dataProvider
        Key Validation Class: org.apache.cassandra.db.marshal.UTF8Type
        Default column value validator: org.apache.cassandra.db.marshal.UTF8Type
        Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type
        Row cache size / save period in seconds / keys to save : 1024.0/0/all
        Row Cache Provider: org.apache.cassandra.cache.SerializingCacheProvider
        Key cache size / save period in seconds: 1024.0/14400
        GC grace seconds: 432000
        Compaction min/max thresholds: 4/32
        Read repair chance: 1.0
        Replicate on write: true
        Column Metadata:
        Column Name: active
        Validation Class: org.apache.cassandra.db.marshal.UTF8Type
        Index Name: dataProvider_active_idx
        Index Type: KEYS
        Column Name: object
        Validation Class: org.apache.cassandra.db.marshal.BytesType
        Column Name: providerData
        Validation Class: org.apache.cassandra.db.marshal.UTF8Type
        Index Name: dataProvider_providerData_idx
        Index Type: KEYS
        Column Name: providerID
        Validation Class: org.apache.cassandra.db.marshal.UTF8Type
        Index Name: dataProvider_providerID_idx
        Index Type: KEYS
        Compaction Strategy: org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy

        Show
        Zhong Li added a comment - There is no exceptions on other nodes. I might be wrong about 'For some column families'. I saw another column family failed with Range Slice too. It works for insert. It might work for others retrieve command. I need test more when I have time. There is no compression, one row data only. ColumnFamily: dataProvider Key Validation Class: org.apache.cassandra.db.marshal.UTF8Type Default column value validator: org.apache.cassandra.db.marshal.UTF8Type Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type Row cache size / save period in seconds / keys to save : 1024.0/0/all Row Cache Provider: org.apache.cassandra.cache.SerializingCacheProvider Key cache size / save period in seconds: 1024.0/14400 GC grace seconds: 432000 Compaction min/max thresholds: 4/32 Read repair chance: 1.0 Replicate on write: true Column Metadata: Column Name: active Validation Class: org.apache.cassandra.db.marshal.UTF8Type Index Name: dataProvider_active_idx Index Type: KEYS Column Name: object Validation Class: org.apache.cassandra.db.marshal.BytesType Column Name: providerData Validation Class: org.apache.cassandra.db.marshal.UTF8Type Index Name: dataProvider_providerData_idx Index Type: KEYS Column Name: providerID Validation Class: org.apache.cassandra.db.marshal.UTF8Type Index Name: dataProvider_providerID_idx Index Type: KEYS Compaction Strategy: org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy
        Hide
        Dominic Williams added a comment -

        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

        Show
        Dominic Williams added a comment - 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
        Hide
        Sylvain Lebresne added a comment -

        This is due to CASSANDRA-3440. More precisely, the fact that in RowRepairResolver it has changed the message from the mutation verb to the read_repair one. The problem is that ReadRepairVerbHandler does not respond anything, but the RowRepairResolver is waiting for a response.

        After looking, I haven't found any part of the code using the read_repair verb handler except for the RowRepairResolver (which would mean that before CASSANDRA-3440 it wasn't used at all, so it's worth having someone else double checking I didn't missed anything), so a simple fix is to make the ReadRepairVerbHandler return an acknowledgment. Attaching a patch for that.

        Show
        Sylvain Lebresne added a comment - This is due to CASSANDRA-3440 . More precisely, the fact that in RowRepairResolver it has changed the message from the mutation verb to the read_repair one. The problem is that ReadRepairVerbHandler does not respond anything, but the RowRepairResolver is waiting for a response. After looking, I haven't found any part of the code using the read_repair verb handler except for the RowRepairResolver (which would mean that before CASSANDRA-3440 it wasn't used at all, so it's worth having someone else double checking I didn't missed anything), so a simple fix is to make the ReadRepairVerbHandler return an acknowledgment. Attaching a patch for that.
        Sylvain Lebresne made changes -
        Field Original Value New Value
        Attachment 3551.patch [ 12506485 ]
        Sylvain Lebresne made changes -
        Status Open [ 1 ] Patch Available [ 10002 ]
        Assignee Sylvain Lebresne [ slebresne ]
        Reviewer jbellis
        Fix Version/s 1.0.6 [ 12319161 ]
        Hide
        Jonathan Ellis added a comment -

        To be explicit: this only affects queries at CL > ONE.

        I haven't found any part of the code using the read_repair verb handler except for the RowRepairResolver, which would mean that before CASSANDRA-3440 it wasn't used at all

        Right, it was used for a while, then we switched to MUTATION Verb to get the reply for "free" when we changed StorageProxy to wait for repair acks, but we never cleared out the Verb or RRVH.

        +1 on the fix.

        Show
        Jonathan Ellis added a comment - To be explicit: this only affects queries at CL > ONE. I haven't found any part of the code using the read_repair verb handler except for the RowRepairResolver, which would mean that before CASSANDRA-3440 it wasn't used at all Right, it was used for a while, then we switched to MUTATION Verb to get the reply for "free" when we changed StorageProxy to wait for repair acks, but we never cleared out the Verb or RRVH. +1 on the fix.
        Hide
        Sylvain Lebresne added a comment -

        Committed, thanks

        Show
        Sylvain Lebresne added a comment - Committed, thanks
        Sylvain Lebresne made changes -
        Status Patch Available [ 10002 ] Resolved [ 5 ]
        Resolution Fixed [ 1 ]
        Hide
        Janne Jalkanen added a comment -

        Confirmed fixed in 1.0.6. Thanks!

        Show
        Janne Jalkanen added a comment - Confirmed fixed in 1.0.6. Thanks!
        Jonathan Ellis made changes -
        Summary Timeout exception for Range Slice after upgrade from 1.0.2 to 1.0.5 Timeout exception for quorum reads after upgrade from 1.0.2 to 1.0.5
        Gavin made changes -
        Workflow no-reopen-closed, patch-avail [ 12644245 ] patch-available, re-open possible [ 12749292 ]
        Gavin made changes -
        Workflow patch-available, re-open possible [ 12749292 ] reopen-resolved, no closed status, patch-avail, testing [ 12754157 ]

          People

          • Assignee:
            Sylvain Lebresne
            Reporter:
            Zhong Li
            Reviewer:
            Jonathan Ellis
          • Votes:
            1 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development