DBZ-8144 Check open redo thread has log with read scn

This commit is contained in:
Chris Cranford 2024-08-13 13:12:52 -04:00 committed by Chris Cranford
parent 9cd78d88a6
commit 8226c2a191
2 changed files with 42 additions and 0 deletions

View File

@ -263,6 +263,13 @@ private boolean isOpenThreadConsistent(RedoThread thread, Scn startScn, List<Log
LOGGER.debug("Redo Thread {} is consistent after enabled SCN {} ({}).", threadId, enabledScn, thread.getStatus());
}
else {
// If thread is open and read position is after enabled position, then redo thread should
// have a log with the read position within its bounds; otherwise it's inconsistent.
if (threadLogs.stream().noneMatch(log -> log.isScnInLogFileRange(startScn))) {
logException(String.format("Redo Thread %d is inconsistent; does not have a log that contains scn %s", threadId, startScn));
return false;
}
final Optional<Long> missingSequence = getFirstLogMissingSequence(threadLogs);
if (missingSequence.isPresent()) {
logException(String.format("Redo Thread %d is inconsistent; failed to find log with sequence %d",

View File

@ -1768,6 +1768,41 @@ public void testRacMultipleNodesOpenedAndAddNewNodeInOpened() throws Exception {
assertThat(collector.getLogs(Scn.ONE)).isEqualTo(files);
}
@Test
@FixFor("DBZ-8144")
public void testOpenRedoThreadSwitchDoesNotMissArchiveLogIfNotYetAvailable() throws Exception {
// In this scenario, a redo log has recently transitioned to the archive, and the entry
// in the V$ARCHIVED_LOG table has not yet appeared; however by all accounts there are
// no log sequence gaps since the only log read is the online redo.
final RedoThreadState redoThreadState = RedoThreadState.builder()
.thread()
.threadId(1)
.enabled("PUBLIC")
.enabledScn(Scn.valueOf(12345))
.status("OPEN")
.lastRedoScn(Scn.valueOf(1234567890))
.lastRedoSequenceNumber(1065L)
.build()
.build();
final List<LogFile> files = new ArrayList<>();
files.add(createRedoLog("redo0", 1234567800, 1065, 1));
final Configuration config = getDefaultConfig().build();
final OracleConnection connection = getOracleConnectionMock(redoThreadState);
// This should be false because the read position is before the redo logs scn
LogFileCollector collector = setCollectorLogFiles(getLogFileCollector(config, connection), files);
assertThat(collector.isLogFileListConsistent(Scn.valueOf(1234567250), files, redoThreadState)).isFalse();
// Now add the archive log to the collected files
files.add(createArchiveLog("arc0", 123456700, 1234567800, 1064, 1));
// This should be true because the read position is within one log
collector = setCollectorLogFiles(getLogFileCollector(config, connection), files);
assertThat(collector.isLogFileListConsistent(Scn.valueOf(1234567250), files, redoThreadState)).isTrue();
}
private static LogFile createRedoLog(String name, long startScn, int sequence, int threadId) {
return createRedoLog(name, startScn, Long.MAX_VALUE, sequence, threadId);
}