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

cassandra-stress tool - InvalidQueryException: Batch too large

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Won't Do
    • 4.0-alpha3, 4.0
    • Legacy/Tools
    • None
    • Cassandra 2.2.4, Java 8, CentOS 6.5

    • Normal

    Description

      I am using Cassandra 2.2.4 and I am struggling to get the cassandra-stress tool to work for my test scenario. I have followed the example on http://www.datastax.com/dev/blog/improved-cassandra-2-1-stress-tool-benchmark-any-schema to create a yaml file describing my test (attached).

      I am collecting events per user id (text, partition key). Events have a session type (text), event type (text), and creation time (timestamp) (clustering keys, in that order). Plus some more attributes required for rendering the events in a UI. For testing purposes I ended up with the following column spec and insert distribution:

      columnspec:
        - name: created_at
          cluster: uniform(10..10000)
        - name: event_type
          size: uniform(5..10)
          population: uniform(1..30)
          cluster: uniform(1..30)
        - name: session_type
          size: fixed(5)
          population: uniform(1..4)
          cluster: uniform(1..4)
        - name: user_id
          size: fixed(15)
          population: uniform(1..1000000)
        - name: message
          size: uniform(10..100)
          population: uniform(1..100B)
      
      insert:
        partitions: fixed(1)
        batchtype: UNLOGGED
        select: fixed(1)/1200000
      

      Running stress tool for just the insert prints

      Generating batches with [1..1] partitions and [0..1] rows (of [10..1200000] total rows in the partitions)
      

      and then immediately starts flooding me with com.datastax.driver.core.exceptions.InvalidQueryException: Batch too large.

      Why I should be exceeding the batch_size_fail_threshold_in_kb: 50 in the cassandra.yaml I do not understand. My understanding is that the stress tool should generate one row per batch. The size of a single row should not exceed 8+10*3+5*3+15*3+100*3 = 398 bytes. Assuming a worst case of all text characters being 3 byte unicode characters.

      This is how I start the attached user scenario:

      [rsteppac@centos bin]$ ./cassandra-stress user profile=../batch_too_large.yaml ops\(insert=1\) -log level=verbose file=~/centos_event_by_patient_session_event_timestamp_insert_only.log -node 10.211.55.8
      INFO  08:00:07 Did not find Netty's native epoll transport in the classpath, defaulting to NIO.
      INFO  08:00:08 Using data-center name 'datacenter1' for DCAwareRoundRobinPolicy (if this is incorrect, please provide the correct datacenter name with DCAwareRoundRobinPolicy constructor)
      INFO  08:00:08 New Cassandra host /10.211.55.8:9042 added
      Connected to cluster: Titan_DEV
      Datatacenter: datacenter1; Host: /10.211.55.8; Rack: rack1
      Created schema. Sleeping 1s for propagation.
      Generating batches with [1..1] partitions and [0..1] rows (of [10..1200000] total rows in the partitions)
      com.datastax.driver.core.exceptions.InvalidQueryException: Batch too large
      	at com.datastax.driver.core.exceptions.InvalidQueryException.copy(InvalidQueryException.java:35)
      	at com.datastax.driver.core.DefaultResultSetFuture.extractCauseFromExecutionException(DefaultResultSetFuture.java:271)
      	at com.datastax.driver.core.DefaultResultSetFuture.getUninterruptibly(DefaultResultSetFuture.java:185)
      	at com.datastax.driver.core.AbstractSession.execute(AbstractSession.java:55)
      	at org.apache.cassandra.stress.operations.userdefined.SchemaInsert$JavaDriverRun.run(SchemaInsert.java:87)
      	at org.apache.cassandra.stress.Operation.timeWithRetry(Operation.java:159)
      	at org.apache.cassandra.stress.operations.userdefined.SchemaInsert.run(SchemaInsert.java:119)
      	at org.apache.cassandra.stress.StressAction$Consumer.run(StressAction.java:309)
      Caused by: com.datastax.driver.core.exceptions.InvalidQueryException: Batch too large
      	at com.datastax.driver.core.Responses$Error.asException(Responses.java:125)
      	at com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:120)
      	at com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:186)
      	at com.datastax.driver.core.RequestHandler.access$2300(RequestHandler.java:45)
      	at com.datastax.driver.core.RequestHandler$SpeculativeExecution.setFinalResult(RequestHandler.java:752)
      	at com.datastax.driver.core.RequestHandler$SpeculativeExecution.onSet(RequestHandler.java:576)
      	at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:1003)
      	at com.datastax.driver.core.Connection$Dispatcher.channelRead0(Connection.java:925)
      	at com.datastax.shaded.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
      	at com.datastax.shaded.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
      	at com.datastax.shaded.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
      	at com.datastax.shaded.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
      	at com.datastax.shaded.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
      	at com.datastax.shaded.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847)
      	at com.datastax.shaded.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
      	at com.datastax.shaded.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
      	at com.datastax.shaded.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
      	at com.datastax.shaded.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
      	at com.datastax.shaded.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
      	at com.datastax.shaded.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
      	at java.lang.Thread.run(Thread.java:745)
      ...
      

      The C* log:

      INFO  08:00:04 Listening for thrift clients...
      WARN  08:00:07 Detected connection using native protocol version 2. Both version 1 and 2 of the native protocol are now deprecated and support will be removed in Cassandra 3.0. You are encouraged to upgrade to a client driver using version 3 of the native protocol
      ERROR 08:00:14 Batch of prepared statements for [stresscql.batch_too_large] is of size 58024, exceeding specified threshold of 51200 by 6824. (see batch_size_fail_threshold_in_kb)
      ERROR 08:00:15 Batch of prepared statements for [stresscql.batch_too_large] is of size 77985, exceeding specified threshold of 51200 by 26785. (see batch_size_fail_threshold_in_kb)
      ...
      

      Attachments

        1. 11105-trunk.txt
          11 kB
          Alexander Dejanovski
        2. batch_too_large.yaml
          1 kB
          Ralf Steppacher

        Issue Links

          Activity

            People

              Unassigned Unassigned
              ralfsteppacher Ralf Steppacher
              Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: