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

Groupby queries trigger ArrayIndexOutOfBoundsException on mixed version cluster

    XMLWordPrintableJSON

Details

    Description

      When I have a mixed cluster with C*3.10 and C*4.0-beta4, issuing GROUP BY query to 3.10 will trigger `java.lang.ArrayIndexOutOfBoundsException`. 

      Reproduce:

      having a mixed cluster 3.10 and 4.0-beta4

       

      // create keyspace and db
      cqlsh> CREATE KEYSPACE test WITH replication = {'class':'SimpleStrategy', 'replication_factor' : 1};
      cqlsh> use test;
      cqlsh:test> create table login_log ( user_id int, application_name text, primary key (user_id, application_name) ) with clustering order by (application_name asc);
      cqlsh:test> insert into login_log (user_id, application_name) VALUES(1, 'bash');cqlsh:test> insert into login_log (user_id, application_name) VALUES(1, 'chrome');
      // issue GROUP BY QUERY
      cqlsh:test> select user_id, application_name from login_log group by user_id, application_name;

       

      The reason why the bug happens is that the Kind enum in DataLimits has changed from 6 values to 4 values: 

       

      [THRIFT_LIMIT, SUPER_COLUMN_COUNTING_LIMIT, CQL_GROUP_BY_LIMIT, CQL_GROUP_BY_PAGING_LIMIT]
      [CQL_LIMIT, CQL_PAGING_LIMIT, CQL_GROUP_BY_LIMIT, CQL_GROUP_BY_PAGING_LIMIT]    
      

      Thus when node 3.10 forwards the read command with CQL_GROUP_BY_LIMIT to node 4.0, it tries to read the value of index 4 in Kind which has only 4 elements, causing ArrayIndexOutOfBoundsException

      Log details:

      ERROR [Messaging-EventLoop-3-5] 2021-04-09 00:27:14,899 InboundMessageHandler.java:334 - /251.250.238.1:7000->/251.250.238.2:7000-LEGACY_MESSAGES-c356fde1 unexpected exception caught while deserializing a message
      java.lang.ArrayIndexOutOfBoundsException: 4
              at org.apache.cassandra.db.filter.DataLimits$Serializer.deserialize(DataLimits.java:1172)
              at org.apache.cassandra.db.ReadCommand$Serializer.deserialize(ReadCommand.java:1006)
              at org.apache.cassandra.db.ReadCommand$Serializer.deserialize(ReadCommand.java:909)
              at org.apache.cassandra.net.Message$Serializer.deserializePayloadPre40(Message.java:966)
              at org.apache.cassandra.net.Message$Serializer.deserializePre40(Message.java:947)
              at org.apache.cassandra.net.Message$Serializer.deserializePre40(Message.java:935)
              at org.apache.cassandra.net.Message$Serializer.deserialize(Message.java:635)
              at org.apache.cassandra.net.InboundMessageHandler.processSmallMessage(InboundMessageHandler.java:320)
              at org.apache.cassandra.net.InboundMessageHandler.processOneContainedMessage(InboundMessageHandler.java:303)
              at org.apache.cassandra.net.InboundMessageHandler.processFrameOfContainedMessages(InboundMessageHandler.java:270)
              at org.apache.cassandra.net.InboundMessageHandler.processIntactFrame(InboundMessageHandler.java:255)
              at org.apache.cassandra.net.InboundMessageHandler.process(InboundMessageHandler.java:246)
              at org.apache.cassandra.net.FrameDecoder.deliver(FrameDecoder.java:320)
              at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:284)
              at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:268)
              at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
              at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
              at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
              at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
              at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
              at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
              at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
              at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:792)
              at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe$1.run(AbstractEpollChannel.java:387)
              at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
              at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
              at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
              at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
              at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
              at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
              at java.lang.Thread.run(Thread.java:748)`
      

       

       

      Attachments

        Activity

          People

            aholmber Adam Holmberg
            jwjwyoung junwen yang
            Adam Holmberg, Benjamin Lerer
            Brandon Williams
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - Not Specified
                Not Specified
                Remaining:
                Remaining Estimate - 0h
                0h
                Logged:
                Time Spent - 20m
                20m