Skip to content

Commit

Permalink
CNDB-13079: Block on validation future in ValidatorTest to ensure lis…
Browse files Browse the repository at this point in the history
…tener has been called. Remove superfluous sleeps.
  • Loading branch information
jkni committed Feb 21, 2025
1 parent 46ec671 commit a07f6b4
Showing 1 changed file with 4 additions and 12 deletions.
16 changes: 4 additions & 12 deletions test/unit/org/apache/cassandra/repair/ValidatorTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;

Expand Down Expand Up @@ -60,11 +61,9 @@
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.MerkleTree;
import org.apache.cassandra.utils.MerkleTrees;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.UUIDGen;

import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.UNIT_TESTS;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
Expand Down Expand Up @@ -195,9 +194,6 @@ public void simpleValidationTest(int n) throws Exception
cfs.forceBlockingFlush(UNIT_TESTS);
assertEquals(1, cfs.getLiveSSTables().size());

// wait enough to force single compaction
TimeUnit.SECONDS.sleep(5);

SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
UUID repairSessionId = UUIDGen.getTimeUUID();
final RepairJobDesc desc = new RepairJobDesc(repairSessionId, UUIDGen.getTimeUUID(), cfs.keyspace.getName(),
Expand All @@ -223,7 +219,7 @@ public void onCompleted(TableOperation.Progress progressOnCompleted)

final CompletableFuture<Message> outgoingMessageSink = registerOutgoingMessageSink();
Validator validator = new Validator(desc, host, 0, true, false, PreviewKind.NONE);
ValidationManager.instance.submitValidation(cfs, validator);
Future<?> validationFuture = ValidationManager.instance.submitValidation(cfs, validator);

Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS);
assertEquals(Verb.VALIDATION_RSP, message.verb());
Expand All @@ -238,6 +234,8 @@ public void onCompleted(TableOperation.Progress progressOnCompleted)
}
assertEquals(m.trees.rowCount(), n);

// block on validation future to ensure the compaction progress listener has been called
validationFuture.get(TEST_TIMEOUT, TimeUnit.SECONDS);
assertNotNull(progressOnCompletion.get());
assertEquals(OperationType.VALIDATION, progressOnCompletion.get().operationType());
assertTrue(progressOnCompletion.get().completed() > 0);
Expand Down Expand Up @@ -268,9 +266,6 @@ public void testSizeLimiting() throws Exception
cfs.forceBlockingFlush(UNIT_TESTS);
assertEquals(1, cfs.getLiveSSTables().size());

// wait enough to force single compaction
TimeUnit.SECONDS.sleep(5);

SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
UUID repairSessionId = UUIDGen.getTimeUUID();
final RepairJobDesc desc = new RepairJobDesc(repairSessionId, UUIDGen.getTimeUUID(), cfs.keyspace.getName(),
Expand Down Expand Up @@ -327,9 +322,6 @@ public void testRangeSplittingTreeSizeLimit() throws Exception
cfs.forceBlockingFlush(UNIT_TESTS);
assertEquals(1, cfs.getLiveSSTables().size());

// wait enough to force single compaction
TimeUnit.SECONDS.sleep(5);

SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
UUID repairSessionId = UUIDGen.getTimeUUID();

Expand Down

0 comments on commit a07f6b4

Please sign in to comment.