From afa05135bc7a357abf7579178de5111131a3db9e Mon Sep 17 00:00:00 2001 From: Raymond Huffman Date: Thu, 21 Nov 2024 18:36:56 -0500 Subject: [PATCH] Add some additional logging to StorageService#drain (#585) --- .../org/apache/cassandra/service/StorageService.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 29d478ae15..190c112b32 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -4635,10 +4635,10 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti ScheduledExecutors.optionalTasks.shutdown(); Gossiper.instance.stop(); - setMode(Mode.DRAINING, "shutting down MessageService", false); + setMode(Mode.DRAINING, "shutting down MessageService", true); MessagingService.instance().shutdown(); - setMode(Mode.DRAINING, "clearing mutation stage", false); + setMode(Mode.DRAINING, "clearing mutation stage", true); counterMutationStage.shutdown(); mutationStage.shutdown(); counterMutationStage.awaitTermination(3600, TimeUnit.SECONDS); @@ -4646,7 +4646,7 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti StorageProxy.instance.verifyNoHintsInProgress(); - setMode(Mode.DRAINING, "flushing column families", false); + setMode(Mode.DRAINING, "flushing column families", true); // count CFs first, since forceFlush could block for the flushWriter to get a queue slot empty totalCFs = 0; for (Keyspace keyspace : Keyspace.nonSystem()) @@ -4668,6 +4668,7 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti remainingCFs--; } + setMode(Mode.DRAINING, "shutdown BatchlogManager", true); try { /* not clear this is reasonable time, but propagated from prior embedded behaviour */ @@ -4678,6 +4679,7 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti logger.error("Batchlog manager timed out shutting down", t); } + setMode(Mode.DRAINING, "shutdown CompactionManager", true); // Interrupt on going compaction and shutdown to prevent further compaction CompactionManager.instance.forceShutdown(); @@ -4686,6 +4688,7 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti // Flush system tables after stopping the batchlog manager and compactions since they both modify // system tables (for example compactions can obsolete sstables and the tidiers in SSTableReader update // system tables, see SSTableReader.GlobalTidy) + setMode(Mode.DRAINING, "flush system tables", true); flushes.clear(); for (Keyspace keyspace : Keyspace.system()) { @@ -4694,12 +4697,14 @@ private synchronized void drainInternal() throws IOException, InterruptedExcepti } FBUtilities.waitOnFutures(flushes); + setMode(Mode.DRAINING, "shutdown Commitlog", true); // whilst we've flushed all the CFs, which will have recycled all completed segments, we want to ensure // there are no segments to replay, so we force the recycling of any remaining (should be at most one) CommitLog.instance.forceRecycleAllSegments("Drain"); CommitLog.instance.shutdownBlocking(); + setMode(Mode.DRAINING, "shutdown non-periodic tasks", true); // wait for miscellaneous tasks like sstable and commitlog segment deletion ScheduledExecutors.nonPeriodicTasks.shutdown(); if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, MINUTES))