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

Sporadic CQL operation timeout due to unconfigured table

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Normal
    • Resolution: Unresolved
    • None
    • Local/Commit Log
    • None
    • Availability
    • Normal
    • Challenging
    • User Report
    • All
    • None

    Description

          We were doing some systematic testing in Cassandra stable release 3.11.10 and found the disk I/O delay may cause some issues.

          We start a Cassandra cluster of 3 nodes with the default configuration, in node 1, we open the cqlsh shell and type in:

      create keyspace ycsb WITH REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor': 3 };
      USE ycsb;
      create table usertable (
          y_id varchar primary key,
          field0 varchar,
          field1 varchar,
          field2 varchar,
          field3 varchar,
          field4 varchar,
          field5 varchar,
          field6 varchar,
          field7 varchar,
          field8 varchar,
          field9 varchar);
      

          During this process, we inject a single disk I/O delay in node 2’s CommitLog#add method (e.g., in Mutation.serializer.serialize).

          /**
           * Add a Mutation to the commit log. If CDC is enabled, this can fail.
           *
           * @param mutation the Mutation to add to the log
           * @throws WriteTimeoutException
           */
          public CommitLogPosition add(Mutation mutation) throws WriteTimeoutException
          {
              assert mutation != null;
      
              try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get())
              {
                  Mutation.serializer.serialize(mutation, dob, MessagingService.current_version);
                  int size = dob.getLength();
                  int totalSize = size + ENTRY_OVERHEAD_SIZE;
                  if (totalSize > MAX_MUTATION_SIZE)
                  {
                      throw new IllegalArgumentException(String.format("Mutation of %s is too large for the maximum size of %s",
                                                                       FBUtilities.prettyPrintMemory(totalSize),
                                                                       FBUtilities.prettyPrintMemory(MAX_MUTATION_SIZE)));
                  }
                  // ...
              }
              catch (IOException e)
              {
                  throw new FSWriteError(e, segmentManager.allocatingFrom().getPath());
              }
          }
      

          In the aforementioned cqlsh shell connected to node 1, when it runs the command of “create table usertable ...”, it costs more time than normal, and finally shows a warning message:

      Warning: schema version mismatch detected; check the schema versions of your nodes in system.local and system.peers.
      

          This behavior seems reasonable because our injection affects node 2.

          When we open another cqlsh shell connecting to node 2, intent to run some commands on the created table:

      USE ycsb;
      

          The cqlsh shell shows an error message:

      InvalidRequest: Error from server: code=2200 [Invalid query] message="unconfigured table t"
      

          It seems reasonable, too, because our injection affects node 2. However, when we run a cqlsh shell connecting to node 1 again, and try to run some commands like:

      USE ycsb;
      SELECT * FROM usertable;
      

          Most of the time, the result is shown immediately. However, sometimes it gets stuck for a few seconds, and shows the error message:

      ReadTimeout: Error from server: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message="Operation timed out - received only 0 responses." info={'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'}
      

          A more reliable way to reproduce this failure is using our client based on datastax (see https://github.com/functioner/datastax-cassandra-client).

          We have dump the jstack for node 2 when the issue happens:

      "MutationStage-1" #90 daemon prio=5 os_prio=0 tid=0x00007f858429c880 nid=0x6f23 waiting on condition [0x00007f856889e000]
         java.lang.Thread.State: TIMED_WAITING (sleeping)
          at java.lang.Thread.sleep(Native Method)
          at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:246)
          at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:593)
          at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:470)
          at org.apache.cassandra.db.Mutation.apply(Mutation.java:250)
          at org.apache.cassandra.db.Mutation.apply(Mutation.java:255)
          at org.apache.cassandra.db.Mutation.apply(Mutation.java:264)
          at org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:597)
          at org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:577)
          at org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:315)
          at org.apache.cassandra.db.SystemKeyspace.lambda$updatePeerInfo$1(SystemKeyspace.java:757)
          at org.apache.cassandra.db.SystemKeyspace$$Lambda$286/119781138.run(Unknown Source)
          at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
          at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
          at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:113)
          at java.lang.Thread.run(Thread.java:748)
      
      "MigrationStage:1" #69 daemon prio=5 os_prio=0 tid=0x00007f857c6d1b20 nid=0x6ecd waiting on condition [0x00007f8568cef000]
         java.lang.Thread.State: WAITING (parking)
          at sun.misc.Unsafe.park(Native Method)
          - parking to wait for  <0x00000005e5e7abd8> (a com.google.common.util.concurrent.ListenableFutureTask)
          at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
          at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
          at java.util.concurrent.FutureTask.get(FutureTask.java:191)
          at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:438)
          at org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$1(SchemaKeyspace.java:310)
          at org.apache.cassandra.schema.SchemaKeyspace$$Lambda$263/1985363058.accept(Unknown Source)
          at java.lang.Iterable.forEach(Iterable.java:75)
          at org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:310)
          at org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1398)
          - locked <0x0000000611bd39e0> (a java.lang.Class for org.apache.cassandra.schema.SchemaKeyspace)
          at org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1380)
          - locked <0x0000000611bd39e0> (a java.lang.Class for org.apache.cassandra.schema.SchemaKeyspace)
          at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51)
          at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
          at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
          at java.util.concurrent.FutureTask.run(FutureTask.java:266)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
          at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
          at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown Source)
          at java.lang.Thread.run(Thread.java:748)
      
      "MemtablePostFlush:1" #22 daemon prio=5 os_prio=0 tid=0x00007f8598b018e0 nid=0x6e45 waiting on condition [0x00007f856e1ad000]
         java.lang.Thread.State: WAITING (parking)
          at sun.misc.Unsafe.park(Native Method)
          - parking to wait for  <0x00000005e5e7abb8> (a java.util.concurrent.CountDownLatch$Sync)
          at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
          at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
          at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
          at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
          at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
          at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1019)
          at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1002)
          at java.util.concurrent.FutureTask.run(FutureTask.java:266)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
          at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
          at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown Source)
          at java.lang.Thread.run(Thread.java:748)
      
      "MemtableFlushWriter:2" #19 daemon prio=5 os_prio=0 tid=0x00007f8598af9da0 nid=0x6e42 waiting on condition [0x00007f856e270000]
         java.lang.Thread.State: WAITING (parking)
          at sun.misc.Unsafe.park(Native Method)
          at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
          at org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:280)
          at org.apache.cassandra.utils.concurrent.OpOrder$Group.await(OpOrder.java:264)
          at org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:431)
          at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1114)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
          at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
          at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown Source)
      

          We found that the delay in CommitLog#add make another thread (DefinitionsUpdateVerbHandler$1.runMayThrow) stuck. In particular, `DefinitionsUpdateVerbHandler` is stuck in waiting for `SchemaKeyspace.flush`, which in turn waits for the `ColumnFamilyStore.waitForFlushes`.

          We are not sure what is the best place to mitigate the issue. One potential fix is to perhaps introduce a timeout limit (default no limit) in the add method, e.g., execute the I/O operations in a Future task and call waitOnFutures with a timeout in the end. If the timeout is exceeded, a WriteTimeoutException would be thrown.

          Any comments and suggestions would be appreciated.

      Attachments

        Activity

          People

            Unassigned Unassigned
            functioner Haoze Wu
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: