This is an automated email from the ASF dual-hosted git repository.
ycai pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push:
new c48906394e Fix testCDCIndexFileWriteOnSync and document cdc index file
read edge case
c48906394e is described below
commit c48906394e01460382f4070ecc34f6f9754fc567
Author: Yifan Cai <[email protected]>
AuthorDate: Thu May 5 14:15:49 2022 -0700
Fix testCDCIndexFileWriteOnSync and document cdc index file read edge case
Patch by Yifan Cai; reviewed by Josh McKenzie for CASSANDRA-17416
---
doc/modules/cassandra/pages/operating/cdc.adoc | 7 +++++-
.../commitlog/CommitLogSegmentManagerCDCTest.java | 29 ++++++++++++++++++----
2 files changed, 30 insertions(+), 6 deletions(-)
diff --git a/doc/modules/cassandra/pages/operating/cdc.adoc
b/doc/modules/cassandra/pages/operating/cdc.adoc
index 4ce9b0f52a..e12decd696 100644
--- a/doc/modules/cassandra/pages/operating/cdc.adoc
+++ b/doc/modules/cassandra/pages/operating/cdc.adoc
@@ -16,12 +16,17 @@ persisted to disk. Upon final segment flush, a second line
with the
human-readable word "COMPLETED" will be added to the _cdc.idx file
indicating that Cassandra has completed all processing on the file.
-We we use an index file rather than just encouraging clients to parse
+We use an index file rather than just encouraging clients to parse
the log realtime off a memory mapped handle as data can be reflected in
a kernel buffer that is not yet persisted to disk. Parsing only up to
the listed offset in the _cdc.idx file will ensure that you only parse
CDC data for data that is durable.
+Please note that in rare chances, e.g. slow disk, it is possible for the
+consumer to read an empty value from the _cdc.idx file because update is
+achieved with first truncating the file then write to the file. In such
+case, the consumer should retry read the index file.
+
A threshold of total disk space allowed is specified in the yaml at
which time newly allocated CommitLogSegments will not allow CDC data
until a consumer parses and removes files from the specified cdc_raw
diff --git
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
index 076f2fbad2..da1586db6c 100644
---
a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
+++
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
@@ -24,7 +24,9 @@ import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.*;
+import java.util.concurrent.TimeUnit;
+import
com.google.monitoring.runtime.instrumentation.common.util.concurrent.Uninterruptibles;
import org.apache.cassandra.io.util.File;
import org.apache.cassandra.io.util.FileReader;
import org.junit.Assert;
@@ -157,11 +159,28 @@ public class CommitLogSegmentManagerCDCTest extends
CQLTester
Assert.assertTrue("Index file not written: " + cdcIndexFile,
cdcIndexFile.exists());
// Read index value and confirm it's == end from last sync
- BufferedReader in = new BufferedReader(new FileReader(cdcIndexFile));
- String input = in.readLine();
- Integer offset = Integer.parseInt(input);
- Assert.assertEquals(syncOffset, (long)offset);
- in.close();
+ String input = null;
+ // There could be a race between index file update (truncate & write)
and read. See CASSANDRA-17416
+ // It is possible to read an empty line. In this case, re-try at most
5 times.
+ for (int i = 0; input == null && i < 5; i++)
+ {
+ if (i != 0) // add a little pause between each attempt
+ Uninterruptibles.sleepUninterruptibly(10,
TimeUnit.MILLISECONDS);
+
+ try (BufferedReader in = new BufferedReader(new
FileReader(cdcIndexFile)))
+ {
+ input = in.readLine();
+ }
+ }
+
+ if (input == null)
+ {
+ Assert.fail("Unable to read the CDC index file after several
attempts");
+ }
+
+ int indexOffset = Integer.parseInt(input);
+ Assert.assertTrue("The offset read from CDC index file should be equal
or larger than the offset after sync. See CASSANDRA-17416",
+ syncOffset <= indexOffset);
}
@Test
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]