Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Fix Version/s: 1.0.9
    • Component/s: None
    • Labels:
      None
    • Environment:

      We start getting this exception after upgrading from 1.0.1 -> 1.0.8.

      4 nodes cluster on Cassandra v1.0.8. RF = 3.
      Hector v0.8.0-3.

      Description

      We have a column family with String row keys and Long column keys.

      Our WideEntityService is trying to get the first column in the range from 0 to Long.MAX. It's a batch operation performed for every row in the CF (rows count is approximately tens of thousands and each row contains from 0 to 1000 columns).

      After processing each row we are removing some of the columns we have queried. Also, at the same time we are writing in this CF in another threads but somewhat less intensive.

      An error rises approximately for a one of 100 rows.

      Exception itself:
      [05-Mar-2012 18:47:25,247] ERROR [http-8095-1 WideEntityServiceImpl.java:142] - get: key1 -

      {type=RANGE, start=0, end=9223372036854775807, orderDesc=false, limit=1}

      me.prettyprint.hector.api.exceptions.HCassandraInternalException: Cassandra encountered an internal error processing this request: TApplicationError type: 6 message:Internal error processing get_slice
      at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:31)
      at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:285)
      at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:268)
      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.getSlice(KeyspaceServiceImpl.java:289)
      at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:53)
      at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:49)
      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.ThriftSliceQuery.execute(ThriftSliceQuery.java:48)

      1. 4000.txt
        1 kB
        Sylvain Lebresne

        Activity

        Transition Time In Source Status Execution Times Last Executer Last Execution Date
        Open Open Patch Available Patch Available
        18h 46m 1 Sylvain Lebresne 06/Mar/12 10:32
        Patch Available Patch Available Resolved Resolved
        23h 41m 1 Sylvain Lebresne 07/Mar/12 10:14
        Aleksey Yeschenko made changes -
        Component/s Core [ 12312978 ]
        Gavin made changes -
        Workflow patch-available, re-open possible [ 12749606 ] reopen-resolved, no closed status, patch-avail, testing [ 12754322 ]
        Gavin made changes -
        Workflow no-reopen-closed, patch-avail [ 12656640 ] patch-available, re-open possible [ 12749606 ]
        Sylvain Lebresne made changes -
        Status Patch Available [ 10002 ] Resolved [ 5 ]
        Reviewer jbellis
        Resolution Fixed [ 1 ]
        Hide
        Sylvain Lebresne added a comment -

        Committed, thanks

        Show
        Sylvain Lebresne added a comment - Committed, thanks
        Hide
        Jonathan Ellis added a comment -

        Either one works for me.

        Show
        Jonathan Ellis added a comment - Either one works for me.
        Hide
        Sylvain Lebresne added a comment -

        I hesitate on that one, but in fact I think that this case can realistically happen when:

        1. either the user requested 1 or 2 columns (like in this ticket). Asking for +1 or *2 doesn't change much.
        2. the row has been deleted but one node didn't got the tombstone yet (or at all). In that case, you mostly want for the RR to send the missed tombstone to the node missing it. However, we can't be sure the retry will arrive after the RR (i.e. we may need a third retry if we're unlucky), and so asking for twice more data from that one node may be less efficient.

        I mean, clearly there is no need to over-think this and I'm good with either +1 and *2 but I'm not really convinced one will be better (nor really worse) than the other in general.

        Show
        Sylvain Lebresne added a comment - I hesitate on that one, but in fact I think that this case can realistically happen when: either the user requested 1 or 2 columns (like in this ticket). Asking for +1 or *2 doesn't change much. the row has been deleted but one node didn't got the tombstone yet (or at all). In that case, you mostly want for the RR to send the missed tombstone to the node missing it. However, we can't be sure the retry will arrive after the RR (i.e. we may need a third retry if we're unlucky), and so asking for twice more data from that one node may be less efficient. I mean, clearly there is no need to over-think this and I'm good with either +1 and *2 but I'm not really convinced one will be better (nor really worse) than the other in general.
        Hide
        Jonathan Ellis added a comment -

        +1, although we might want to query for 2*count instead of count+1 in the no-live-columns case, since if all count columns were dead the odds seem pretty good that just one more column won't help.

        Show
        Jonathan Ellis added a comment - +1, although we might want to query for 2*count instead of count+1 in the no-live-columns case, since if all count columns were dead the odds seem pretty good that just one more column won't help.
        Hide
        Sylvain Lebresne added a comment -

        I've just pushed a dtest to reproduce (and checked the patch does fix). It's during short reads, so not sure there's an easy way to test that with a unit test.

        Show
        Sylvain Lebresne added a comment - I've just pushed a dtest to reproduce (and checked the patch does fix). It's during short reads, so not sure there's an easy way to test that with a unit test.
        Hide
        Jonathan Ellis added a comment -

        Can we add a test to catch this regression?

        Show
        Jonathan Ellis added a comment - Can we add a test to catch this regression?
        Sylvain Lebresne made changes -
        Status Open [ 1 ] Patch Available [ 10002 ]
        Fix Version/s 1.0.9 [ 12319856 ]
        Sylvain Lebresne made changes -
        Attachment 4000.txt [ 12517216 ]
        Hide
        Sylvain Lebresne added a comment -

        Slightly stupid bug introduced by CASSANDRA-3934. Fix attached

        Show
        Sylvain Lebresne added a comment - Slightly stupid bug introduced by CASSANDRA-3934 . Fix attached
        Sylvain Lebresne made changes -
        Assignee Sylvain Lebresne [ slebresne ]
        Hide
        Sergey B added a comment -

        ERROR [Thrift:104] 2012-03-01 20:19:57,332 Cassandra.java (line 3041) Internal error processing get_slice
        java.lang.ArithmeticException: / by zero
        at org.apache.cassandra.db.SliceFromReadCommand.maybeGenerateRetryCommand(SliceFromReadCommand.java:87)
        at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:724)
        at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:564)
        at org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:128)
        at org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:283)
        at org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:365)
        at org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:326)
        at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.process(Cassandra.java:3033)
        at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2889)
        at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:187)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)

        Show
        Sergey B added a comment - ERROR [Thrift:104] 2012-03-01 20:19:57,332 Cassandra.java (line 3041) Internal error processing get_slice java.lang.ArithmeticException: / by zero at org.apache.cassandra.db.SliceFromReadCommand.maybeGenerateRetryCommand(SliceFromReadCommand.java:87) at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:724) at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:564) at org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:128) at org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:283) at org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:365) at org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:326) at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.process(Cassandra.java:3033) at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2889) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:187) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662)
        Hide
        amorton added a comment -

        What was the error in the server log ?

        Show
        amorton added a comment - What was the error in the server log ?
        Sergey B made changes -
        Field Original Value New Value
        Environment We start getting this exception after upgrading from 1.0.1 -> 1.0.8.

        4 nodes cluster on Cassandra v1.0.8. RF = 3.
        Hector v0.8.0-3.

        We have a column family with String row keys and Long column keys.

        Our WideEntityService is trying to get the first column in the range from 0 to Long.MAX. It's a batch operation performed for every row in the CF (rows count is approximately tens of thousands and each row contains from 0 to 1000 columns).

        After processing each row we are removing some of the columns we have queried. Also, at the same time we are writing in this CF in another threads but somewhat less intensive.

        An error rises approximately for a one of 100 rows.
        We start getting this exception after upgrading from 1.0.1 -> 1.0.8.

        4 nodes cluster on Cassandra v1.0.8. RF = 3.
        Hector v0.8.0-3.
        Description [05-Mar-2012 18:47:25,247] ERROR [http-8095-1 WideEntityServiceImpl.java:142] - get: key1 - {type=RANGE, start=0, end=9223372036854775807, orderDesc=false, limit=1}
        me.prettyprint.hector.api.exceptions.HCassandraInternalException: Cassandra encountered an internal error processing this request: TApplicationError type: 6 message:Internal error processing get_slice
                at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:31)
                at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:285)
                at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:268)
                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.getSlice(KeyspaceServiceImpl.java:289)
                at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:53)
                at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:49)
                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.ThriftSliceQuery.execute(ThriftSliceQuery.java:48)
        We have a column family with String row keys and Long column keys.

        Our WideEntityService is trying to get the first column in the range from 0 to Long.MAX. It's a batch operation performed for every row in the CF (rows count is approximately tens of thousands and each row contains from 0 to 1000 columns).

        After processing each row we are removing some of the columns we have queried. Also, at the same time we are writing in this CF in another threads but somewhat less intensive.

        An error rises approximately for a one of 100 rows.


        Exception itself:
        [05-Mar-2012 18:47:25,247] ERROR [http-8095-1 WideEntityServiceImpl.java:142] - get: key1 - {type=RANGE, start=0, end=9223372036854775807, orderDesc=false, limit=1}
        me.prettyprint.hector.api.exceptions.HCassandraInternalException: Cassandra encountered an internal error processing this request: TApplicationError type: 6 message:Internal error processing get_slice
                at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:31)
                at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:285)
                at me.prettyprint.cassandra.service.KeyspaceServiceImpl$7.execute(KeyspaceServiceImpl.java:268)
                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.getSlice(KeyspaceServiceImpl.java:289)
                at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:53)
                at me.prettyprint.cassandra.model.thrift.ThriftSliceQuery$1.doInKeyspace(ThriftSliceQuery.java:49)
                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.ThriftSliceQuery.execute(ThriftSliceQuery.java:48)
        Sergey B created issue -

          People

          • Assignee:
            Sylvain Lebresne
            Reporter:
            Sergey B
            Reviewer:
            Jonathan Ellis
          • Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development