Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-9559

IndexOutOfBoundsException inserting into TupleType

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • 2.1.8, 2.2.0 rc2
    • Legacy/CQL
    • None
    • Normal

    Description

      Trying to run this query on the 2.2 branch resulted in IndexOutOfBoundsException:

      INSERT INTO datatypes.alltypes (pk,c_tuple) VALUES (1,'01:02:03.456789012')
      

      The column c_tuple is of type tuple<int>.

      The cause seems to be that TupleType.fromString is splitting the source string on : (meant for UDT only?) but does not consider that the number of those fields may exceed the number of types defined in the Tuple.

      A simple fix would be to bound the for loop with types.size() instead of fieldStrings.size() but still some error should be raised if fieldStrings.size() > types.size().

      java.lang.IndexOutOfBoundsException: Index: 1, Size: 1
              at java.util.ArrayList.rangeCheck(ArrayList.java:635) ~[na:1.7.0_51]
              at java.util.ArrayList.get(ArrayList.java:411) ~[na:1.7.0_51]
              at org.apache.cassandra.db.marshal.TupleType.type(TupleType.java:60) ~[main/:na]
              at org.apache.cassandra.db.marshal.TupleType.fromString(TupleType.java:224) ~[main/:na]
              at org.apache.cassandra.cql3.Constants$Literal.parsedValue(Constants.java:152) ~[main/:na]
              at org.apache.cassandra.cql3.Constants$Literal.prepare(Constants.java:138) ~[main/:na]
              at org.apache.cassandra.cql3.Constants$Literal.prepare(Constants.java:91) ~[main/:na]
              at org.apache.cassandra.cql3.Operation$SetValue.prepare(Operation.java:167) ~[main/:na]
              at org.apache.cassandra.cql3.statements.UpdateStatement$ParsedInsert.prepareInternal(UpdateStatement.java:213) ~[main/:na]
              at org.apache.cassandra.cql3.statements.ModificationStatement$Parsed.prepare(ModificationStatement.java:710) ~[main/:na]
              at org.apache.cassandra.cql3.statements.ModificationStatement$Parsed.prepare(ModificationStatement.java:698) ~[main/:na]
              at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:493) ~[main/:na]
              at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:247) ~[main/:na]
              at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:241) ~[main/:na]
              at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) ~[main/:na]
              at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507) [main/:na]
              at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401) [main/:na]
              at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.23.Final.jar:4.0.23.Final]
              at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) [netty-all-4.0.23.Final.jar:4.0.23.Final]
              at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) [netty-all-4.0.23.Final.jar:4.0.23.Final]
              at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) [netty-all-4.0.23.Final.jar:4.0.23.Final]
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_51]
              at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) [main/:na]
              at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
              at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
      

      Attachments

        1. 9559-2.txt
          4 kB
          Benjamin Lerer
        2. 9559.txt
          3 kB
          Benjamin Lerer

        Activity

          People

            blerer Benjamin Lerer
            djatnieks dan jatnieks
            Benjamin Lerer
            Tom Hobbs
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: