diff --git a/src/java/org/apache/cassandra/metrics/SamplingManager.java b/src/java/org/apache/cassandra/metrics/SamplingManager.java index 37d8d355c2b8..862b81fe772f 100644 --- a/src/java/org/apache/cassandra/metrics/SamplingManager.java +++ b/src/java/org/apache/cassandra/metrics/SamplingManager.java @@ -195,12 +195,6 @@ private Runnable createSamplingBeginRunnable(JobId jobId, Iterable { - if (cancelingTasks.contains(jobId)) - { - logger.debug("The sampling job of {} is currently canceling. Not issuing a new run.", jobId); - activeSamplingTasks.remove(jobId); - return; - } List tableNames = StreamSupport.stream(tables.spliterator(), false) .map(cfs -> String.format("%s.%s", cfs.keyspace, cfs.name)) .collect(Collectors.toList()); diff --git a/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java b/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java index d9cfdeecef91..d6fc06568517 100644 --- a/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java +++ b/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java @@ -31,7 +31,6 @@ import javax.management.openmbean.CompositeData; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.Uninterruptibles; import org.junit.BeforeClass; import org.junit.Test; @@ -43,6 +42,8 @@ import org.apache.cassandra.metrics.Sampler; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.Util; + import static java.lang.String.format; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; @@ -217,11 +218,7 @@ public void testStartAndStopScheduledSampling() assertTrue("Existing scheduled sampling tasks should be cancellable", ss.stopSamplingPartitions(null, null)); - int timeout = 10; - while (timeout-- > 0 && ss.getSampleTasks().size() > 0) - Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); - - assertEquals("Scheduled sampled tasks should be removed", Collections.emptyList(), ss.getSampleTasks()); + Util.spinAssertEquals(Collections.emptyList(), ss::getSampleTasks, 30); assertTrue("When nothing is scheduled, you should be able to stop all scheduled sampling tasks", ss.stopSamplingPartitions(null, null));