From 70f2ef90de2ef8ebfe8a4480d6f7192811b9c913 Mon Sep 17 00:00:00 2001 From: Brandon Williams Date: Wed, 13 May 2026 11:26:48 -0500 Subject: [PATCH] CNDB-17791: measure only active memtable cleanup windows in FlushFailingOnNotificationSubscriberTest Track maxTimeSinceCleanup only while the memory pool actually needs cleaning, and reset the window once cleanup completes or after a successful flush. --- ...shFailingOnNotificationSubscriberTest.java | 27 +++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/test/long/org/apache/cassandra/io/memtable/FlushFailingOnNotificationSubscriberTest.java b/test/long/org/apache/cassandra/io/memtable/FlushFailingOnNotificationSubscriberTest.java index 8b13baf5ed54..5f5886463790 100644 --- a/test/long/org/apache/cassandra/io/memtable/FlushFailingOnNotificationSubscriberTest.java +++ b/test/long/org/apache/cassandra/io/memtable/FlushFailingOnNotificationSubscriberTest.java @@ -62,7 +62,7 @@ public class FlushFailingOnNotificationSubscriberTest extends CQLTester private final AtomicInteger numFailedFlushes = new AtomicInteger(); volatile long maxTimeSinceCleanup = 0; - volatile long lastTimePoolNeededCleaning = System.nanoTime(); + volatile long lastTimePoolNeededCleaning = 0; static AtomicDouble failFlushProbability = new AtomicDouble(FLUSH_FAILURE_PROBABILITY); @@ -116,14 +116,7 @@ public void flushFailingOnSSTableAddingNotificationVSWritesTest() throws Interru } idx++; - if (MEMORY_POOL.needsCleaning()) - { - lastTimePoolNeededCleaning = System.nanoTime(); - } - else - { - updateMaxTimeSinceCleanup(); - } + updateMaxTimeSinceCleanup(); if (MEMORY_POOL.getNumPendingtasks() > 2) { @@ -164,7 +157,19 @@ public void flushFailingOnSSTableAddingNotificationVSWritesTest() throws Interru private void updateMaxTimeSinceCleanup() { - maxTimeSinceCleanup = Math.max(maxTimeSinceCleanup, (System.nanoTime() - lastTimePoolNeededCleaning) / TimeUnit.MILLISECONDS.toNanos(1)); + long now = System.nanoTime(); + if (MEMORY_POOL.needsCleaning()) + { + if (lastTimePoolNeededCleaning == 0) + lastTimePoolNeededCleaning = now; + + maxTimeSinceCleanup = Math.max(maxTimeSinceCleanup, (now - lastTimePoolNeededCleaning) / TimeUnit.MILLISECONDS.toNanos(1)); + } + else if (lastTimePoolNeededCleaning != 0) + { + maxTimeSinceCleanup = Math.max(maxTimeSinceCleanup, (now - lastTimePoolNeededCleaning) / TimeUnit.MILLISECONDS.toNanos(1)); + lastTimePoolNeededCleaning = 0; + } } private void logState() @@ -209,7 +214,7 @@ private void successfulUserFlush() { failFlushProbability.set(0.0); getCurrentColumnFamilyStore().forceFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS).get(); - lastTimePoolNeededCleaning = System.nanoTime(); + lastTimePoolNeededCleaning = 0; failFlushProbability.set(FLUSH_FAILURE_PROBABILITY); } catch (InterruptedException | ExecutionException e)