Skip to content

Commit cdf443e

Browse files
pkolaczkdriftx
authored andcommitted
CNDB-10308: Fix flaky BinLogTest (#1840)
The original test was timing dependent and assumed segments were rolled while the code was waiting in Thread.sleep. A hiccup in performance could violate that assumption and cause rolling the segments at a different point in time, leading to the deleted segment contain fewer entries than expected by the test, and hence leaving more entries in the current segment. The test has been rewritten to not rely on timing at all. Byteman injections are used to detect that the deletion of rolled segment happened. The test doesn't require a particular exact number of entries to be left in the current segment, but checks the total number of segments and the minimum number of entries instead.
1 parent d313862 commit cdf443e

File tree

2 files changed

+67
-35
lines changed

2 files changed

+67
-35
lines changed

test/unit/org/apache/cassandra/Util.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -158,6 +158,7 @@
158158
import org.apache.cassandra.utils.Throwables;
159159
import org.assertj.core.api.Assertions;
160160
import org.awaitility.Awaitility;
161+
import org.awaitility.core.ThrowingRunnable;
161162
import org.hamcrest.Matcher;
162163
import org.mockito.Mockito;
163164
import org.mockito.internal.stubbing.defaultanswers.ForwardsInvocations;
@@ -756,6 +757,18 @@ public static <T> void spinAssert(String message, Matcher<T> matcher, Supplier<?
756757
.untilAsserted(() -> assertThat(message, actualSupplier.get(), matcher));
757758
}
758759

760+
public static void spinAssert(String message, ThrowingRunnable assertion, long timeout, TimeUnit timeUnit) {
761+
Awaitility.await()
762+
.pollInterval(Duration.ofMillis(100))
763+
.pollDelay(0, TimeUnit.MILLISECONDS)
764+
.atMost(timeout, timeUnit)
765+
.untilAsserted(assertion);
766+
}
767+
768+
public static void spinAssert(ThrowingRunnable assertion, int timeoutInSeconds) {
769+
spinAssert(null, assertion, timeoutInSeconds, TimeUnit.SECONDS);
770+
}
771+
759772
public static void joinThread(Thread thread) throws InterruptedException
760773
{
761774
thread.join(10000);

test/unit/org/apache/cassandra/utils/binlog/BinLogTest.java

Lines changed: 54 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,16 @@
2222
import java.nio.file.Files;
2323
import java.util.ArrayList;
2424
import java.util.List;
25+
import java.util.Objects;
2526
import java.util.concurrent.CountDownLatch;
2627
import java.util.concurrent.Semaphore;
2728
import java.util.concurrent.TimeUnit;
2829
import java.util.concurrent.atomic.AtomicInteger;
2930
import java.util.function.Supplier;
3031

3132
import org.apache.cassandra.config.DatabaseDescriptor;
33+
import org.apache.cassandra.inject.Injections;
34+
import org.apache.cassandra.inject.InvokePointBuilder;
3235
import org.apache.cassandra.io.util.File;
3336
import org.junit.After;
3437
import org.junit.Before;
@@ -37,15 +40,18 @@
3740

3841
import net.openhft.chronicle.queue.ChronicleQueue;
3942
import net.openhft.chronicle.queue.ExcerptTailer;
40-
import net.openhft.chronicle.queue.RollCycles;
43+
import net.openhft.chronicle.queue.rollcycles.TestRollCycles;
4144
import net.openhft.chronicle.queue.impl.single.SingleChronicleQueueBuilder;
4245
import net.openhft.chronicle.wire.WireOut;
4346
import org.apache.cassandra.Util;
4447

48+
import static org.apache.cassandra.Util.spinAssert;
49+
import static org.assertj.core.api.Assertions.assertThat;
4550
import static org.apache.cassandra.utils.Clock.Global.nanoTime;
4651
import static org.junit.Assert.assertEquals;
4752
import static org.junit.Assert.assertFalse;
4853
import static org.junit.Assert.assertNotEquals;
54+
import static org.junit.Assert.assertNotSame;
4955
import static org.junit.Assert.assertTrue;
5056
import static org.junit.Assert.fail;
5157

@@ -57,11 +63,16 @@ public static Path tempDir() throws Exception
5763
}
5864

5965
private static final String testString = "ry@nlikestheyankees";
60-
private static final String testString2 = testString + "1";
66+
private static final String testString2 = testString + '1';
6167

6268
private BinLog binLog;
6369
private Path path;
6470

71+
static
72+
{
73+
DatabaseDescriptor.daemonInitialization(); // needed for Injections to work
74+
}
75+
6576
@BeforeClass
6677
public static void setup()
6778
{
@@ -74,7 +85,7 @@ public void setUp() throws Exception
7485
{
7586
path = tempDir();
7687
binLog = new BinLog.Builder().path(path)
77-
.rollCycle(RollCycles.TEST_SECONDLY.toString())
88+
.rollCycle(TestRollCycles.TEST_SECONDLY.toString())
7889
.maxQueueWeight(10)
7990
.maxLogSize(1024 * 1024 * 128)
8091
.blocking(false)
@@ -109,13 +120,13 @@ public void testConstructorNullRollCycle() throws Exception
109120
@Test(expected = IllegalArgumentException.class)
110121
public void testConstructorZeroWeight() throws Exception
111122
{
112-
new BinLog.Builder().path(tempDir()).rollCycle(RollCycles.TEST_SECONDLY.toString()).maxQueueWeight(0).build(false);
123+
new BinLog.Builder().path(tempDir()).rollCycle(TestRollCycles.TEST_SECONDLY.toString()).maxQueueWeight(0).build(false);
113124
}
114125

115126
@Test(expected = IllegalArgumentException.class)
116127
public void testConstructorLogSize() throws Exception
117128
{
118-
new BinLog.Builder().path(tempDir()).rollCycle(RollCycles.TEST_SECONDLY.toString()).maxLogSize(0).build(false);
129+
new BinLog.Builder().path(tempDir()).rollCycle(TestRollCycles.TEST_SECONDLY.toString()).maxLogSize(0).build(false);
119130
}
120131

121132
/**
@@ -166,7 +177,7 @@ public void writeMarshallablePayload(WireOut wire)
166177
});
167178
t.start();
168179
t.join(60 * 1000);
169-
assertEquals("BinLog should not take more than 1 minute to stop", t.getState(), Thread.State.TERMINATED);
180+
assertEquals("BinLog should not take more than 1 minute to stop", Thread.State.TERMINATED, t.getState());
170181

171182
Util.spinAssertEquals(2, releaseCount::get, 60);
172183
Util.spinAssertEquals(Thread.State.TERMINATED, binLog.binLogThread::getState, 60);
@@ -294,7 +305,7 @@ public void writeMarshallablePayload(WireOut wire)
294305
t.start();
295306
Thread.sleep(500);
296307
//If the thread is not terminated then it is probably blocked on the queue
297-
assertTrue(t.getState() != Thread.State.TERMINATED);
308+
assertNotSame(Thread.State.TERMINATED, t.getState());
298309
}
299310
finally
300311
{
@@ -382,7 +393,7 @@ public void writeMarshallablePayload(WireOut wire)
382393
public void testCleanupOnOversize() throws Exception
383394
{
384395
tearDown();
385-
binLog = new BinLog.Builder().path(path).rollCycle(RollCycles.TEST_SECONDLY.toString()).maxQueueWeight(1).maxLogSize(10000).blocking(false).build(false);
396+
binLog = new BinLog.Builder().path(path).rollCycle(TestRollCycles.TEST_SECONDLY.toString()).maxQueueWeight(1).maxLogSize(10000).blocking(false).build(false);
386397
for (int ii = 0; ii < 5; ii++)
387398
{
388399
binLog.put(record(String.valueOf(ii)));
@@ -422,48 +433,56 @@ public void testPutAfterStop() throws Exception
422433

423434
/**
424435
* Test for a bug where files were deleted but the space was not reclaimed when tracking so
425-
* all log segemnts were incorrectly deleted when rolled.
426-
*
427-
* Due to some internal state in ChronicleQueue this test is occasionally
428-
* flaky when run in the suite with testPut or testOffer.
436+
* all log segments were incorrectly deleted when rolled.
429437
*/
430438
@Test
431-
public void testTruncationReleasesLogSpace() throws Exception
439+
public void testTruncationReleasesLogSpace() throws Throwable
432440
{
433-
Util.flakyTest(this::flakyTestTruncationReleasesLogSpace, 2, "Fails occasionally due to Chronicle internal state, see CASSANDRA-16526");
434-
}
435-
436-
437-
private void flakyTestTruncationReleasesLogSpace()
438-
{
439-
StringBuilder sb = new StringBuilder();
440441
try
441442
{
442-
for (int ii = 0; ii < 1024 * 1024 * 2; ii++)
443+
Injections.Counter deletionCounter = Injections.newCounter("binlogSegmentDeletion")
444+
.add(InvokePointBuilder.newInvokePoint()
445+
.onClass("org.apache.cassandra.utils.binlog.DeletingArchiver")
446+
.onMethod("onReleased")
447+
.atExit())
448+
.build();
449+
Injections.inject(deletionCounter);
450+
deletionCounter.reset();
451+
deletionCounter.enable();
452+
453+
String queryString = "a".repeat(1024 * 1024 * 2);
454+
455+
int maxFileCount = 0;
456+
457+
// This should fill up the log so when it rolls in the future it will always delete the rolled segment;
458+
// Make sure we don't delete more than one segment.
459+
// This loop should complete in less than 3 seconds as we're rolling the log every second.
460+
long startTime = System.currentTimeMillis();
461+
while (deletionCounter.get() <= 1 && System.currentTimeMillis() - startTime < 10 * 1000)
443462
{
444-
sb.append('a');
445-
}
463+
binLog.put(record(queryString));
446464

447-
String queryString = sb.toString();
465+
int fileCount = getFileCount(path);
466+
assertThat(fileCount).isGreaterThanOrEqualTo(maxFileCount - 1);
467+
assertThat(fileCount).isLessThanOrEqualTo(3);
468+
maxFileCount = Math.max(maxFileCount, fileCount);
448469

449-
//This should fill up the log so when it rolls in the future it will always delete the rolled segment;
450-
for (int ii = 0; ii < 129; ii++)
451-
{
452-
binLog.put(record(queryString));
470+
spinAssert(() -> assertThat(readBinLogRecords(path).size()).isGreaterThanOrEqualTo(1), 10);
453471
}
454472

455-
for (int ii = 0; ii < 2; ii++)
456-
{
457-
Thread.sleep(2000);
458-
binLog.put(record(queryString));
459-
}
473+
assertThat(deletionCounter.get()).isGreaterThan(0);
474+
assertThat(getFileCount(path)).isGreaterThanOrEqualTo(maxFileCount - 1);
475+
spinAssert(() -> assertThat(readBinLogRecords(path).size()).isGreaterThanOrEqualTo(1), 10);
460476
}
461477
catch (InterruptedException e)
462478
{
463479
throw new RuntimeException(e);
464480
}
481+
}
465482

466-
Util.spinAssertEquals(2, () -> readBinLogRecords(path).size(), 60);
483+
static int getFileCount(Path path)
484+
{
485+
return Objects.requireNonNull(path.toFile().listFiles()).length;
467486
}
468487

469488
static BinLog.ReleaseableWriteMarshallable record(String text)
@@ -495,7 +514,7 @@ public void writeMarshallablePayload(WireOut wire)
495514
List<String> readBinLogRecords(Path path)
496515
{
497516
List<String> records = new ArrayList<String>();
498-
try (ChronicleQueue queue = SingleChronicleQueueBuilder.single(path.toFile()).rollCycle(RollCycles.TEST_SECONDLY).build())
517+
try (ChronicleQueue queue = SingleChronicleQueueBuilder.single(path.toFile()).rollCycle(TestRollCycles.TEST_SECONDLY).build())
499518
{
500519
ExcerptTailer tailer = queue.createTailer();
501520
while (true)

0 commit comments

Comments
 (0)