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

Schema mutations may not be completed on drain

    XMLWordPrintableJSON

Details

    • Degradation
    • Low
    • Normal
    • DTest
    • All
    • None
    • Hide

      Run existing test suites - adding a ByteBuddy test to get the gossiper thread stuck could be a bit brittle.

      Show
      Run existing test suites - adding a ByteBuddy test to get the gossiper thread stuck could be a bit brittle.

    Description

      The drain logic (invoked explicitly with nodetool or from the JVM
      shutdown hook) closes down executor stages that can create mutations (counter,
      view, mutation) before closing down the commitlog. The gossip
      stage also commits schema mutations, and should be treated the same way.

      The messaging service is shut down as part of drain, so there should be
      no new Gossip messages received, however any messages still queued
      in the executor could still run after the commitlog allocator is shut down as
      part of drain, causing the gossip stage thread to hang indefinitely waiting
      for a new segment that never arrives.

      Here is an example from an in-JVM dtest, showing an update to the peers table as it shuts down.

      park:-1, Unsafe (jdk.internal.misc)
      park:323, LockSupport (java.util.concurrent.locks)
      await:289, WaitQueue$Standard$AbstractSignal (org.apache.cassandra.utils.concurrent)
      await:282, WaitQueue$Standard$AbstractSignal (org.apache.cassandra.utils.concurrent)
      awaitUninterruptibly:186, Awaitable$Defaults (org.apache.cassandra.utils.concurrent)
      awaitUninterruptibly:259, Awaitable$AbstractAwaitable (org.apache.cassandra.utils.concurrent)
      awaitAvailableSegment:283, AbstractCommitLogSegmentManager (org.apache.cassandra.db.commitlog)
      advanceAllocatingFrom:257, AbstractCommitLogSegmentManager (org.apache.cassandra.db.commitlog)
      allocate:55, CommitLogSegmentManagerStandard (org.apache.cassandra.db.commitlog)
      add:282, CommitLog (org.apache.cassandra.db.commitlog)
      beginWrite:50, CassandraKeyspaceWriteHandler (org.apache.cassandra.db)
      applyInternal:622, Keyspace (org.apache.cassandra.db)
      apply:506, Keyspace (org.apache.cassandra.db)
      apply:215, Mutation (org.apache.cassandra.db)
      apply:220, Mutation (org.apache.cassandra.db)
      apply:229, Mutation (org.apache.cassandra.db)
      executeInternalWithoutCondition:644, ModificationStatement (org.apache.cassandra.cql3.statements)
      executeLocally:635, ModificationStatement (org.apache.cassandra.cql3.statements)
      executeInternal:431, QueryProcessor (org.apache.cassandra.cql3)
      updateTokens:804, SystemKeyspace (org.apache.cassandra.db)
      updateTokenMetadata:2941, StorageService (org.apache.cassandra.service)
      handleStateNormal:3057, StorageService (org.apache.cassandra.service)
      onChange:2498, StorageService (org.apache.cassandra.service)
      markAsShutdown:607, Gossiper (org.apache.cassandra.gms)
      doVerb:39, GossipShutdownVerbHandler (org.apache.cassandra.gms)
      lambda$new$0:78, InboundSink (org.apache.cassandra.net)
      accept:-1, 581110313 (org.apache.cassandra.net.InboundSink$$Lambda$2638)
      accept:64, InboundSink$Filtered (org.apache.cassandra.net)
      accept:50, InboundSink$Filtered (org.apache.cassandra.net)
      accept:97, InboundSink (org.apache.cassandra.net)
      accept:45, InboundSink (org.apache.cassandra.net)
      run:433, InboundMessageHandler$ProcessMessage (org.apache.cassandra.net)
      run:124, ExecutionFailure$1 (org.apache.cassandra.concurrent)
      runWorker:1128, ThreadPoolExecutor (java.util.concurrent)
      run:628, ThreadPoolExecutor$Worker (java.util.concurrent)
      run:30, FastThreadLocalRunnable (io.netty.util.concurrent)
      run:829, Thread (java.lang)
      

      This causes an exception during shutdown for the in-JVM dtest as it is
      unable to shutdown Stage.GOSSIP, but does not prevent regular
      shutdown for Cassandra as the executors are not stopped. The schema update
      would be lost, despite requesting a graceful shutdown.

      Attachments

        Activity

          People

            jonmeredith Jon Meredith
            jonmeredith Jon Meredith
            Jon Meredith
            Caleb Rackliffe
            Votes:
            0 Vote for this issue
            Watchers:
            3 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 - 2.5h
                2.5h