Chances are if you’re using the Debezium for Oracle connector, you’ve encountered the infamous exception about the SCN could not be found in your transaction logs. In this blog post, we’ll not only talk about what this exception means, but the why, and troubleshooting steps you should take.

What you should know…​

Before we talk about what the exception means and how to troubleshoot it, it’s important to have some understanding of some of the technical details we’re going to be using later in the post.

What are SCNs?

System Change Numbers (SCNs) are Oracle’s internal mechanism for ordering changes. You can think of them as logical timestamps used to track the progression of changes across the database. Every commit, every change to a table, every DDL operation — all get assigned an SCN.

Important notes to consider about SCNs

  • They are monotonically increasing, but not globally unique across contexts. For example, in Oracle RAC, multiple redo threads (which we’ll discuss later) can have overlapping SCN ranges. In addition, Oracle is permitted to assign the same SCN to multiple changes.

  • They are crucial to Oracle LogMiner because Oracle uses SCNs to locate the start and end points of a mining session.

  • Debezium uses SCNs as a resume position during streaming, ensuring that changes are not lost or duplicated across connector restarts.

Redo Logs — Online versus Archived

Oracle writes every transaction into transaction logs called redo logs. These logs come in two flavors:

Online Redo Logs

These are active, circular logs written in real-time by the Oracle Log Writer (LGWR) process. They are finite in size and are eventually overwritten once all changes within them are safely archived.

Archived Redo Logs

When archiving is enabled, Oracle copies full online redo logs to archive destinations before they are overwritten. These are essential for things like:

  • Long-running queries or transactions

  • Point-in-time recovery

  • Downstream systems like Debezium that need historical log data

Redo Threads

Whether you’ve deployed Oracle as a Standalone instance or if you’re using Oracle RAC, each Oracle instance writes to its own redo thread. While SCNs increment globally, each redo thread maintains a logical series of chronological changes that have occurred on that specific instance. In practice, this means:

  • One redo thread (T1) may contain changes from transactions that logically happen after those in another redo thread (T2), even if the SCNs seem interleaved.

  • Debezium and by extension LogMiner, needs access to logs across all redo threads to correctly reconstruct the global order of transactions in the database.

  • Missing or unarchived redo logs for any redo thread can cause gaps, which Debezium guards against by performing log consistency check when Oracle performs a log switch (which we’ll discuss more below)

How to troubleshoot the "SCN not found" error?

There are several points during the Debezium connector lifecycle where the Oracle connector may report that it cannot find a specific SCN in the logs. Let’s break down where this happens and how to troubleshoot each scenario.

Log position validation

When a Debezium Oracle connector starts, one of the first steps is validating that the resume position, based on the last known SCN in the offset, is still valid. To do this, Debezium compares the stored offset SCN against the earliest available SCN in the database’s redo logs (online or archived).

The logic is straightforward:

  • If the earliest SCN ≤ offset SCN, the position is valid.

  • If the earliest SCN > offset SCN, the position is invalid, and Debezium throws an error stating the SCN cannot be found.

Why would the SCN become invalid?

There are several common causes:

Short archive log retention

The Oracle archive policy may be purging logs faster than Debezium can process them.

Extended connector downtime

If the connector is offline longer than the log retention period, the required logs may no longer exist.

Skewed activity between captured and non-captured tables

The global SCN increases as transactions occur, even if Debezium isn’t capturing those tables. This can result in a stale SCN in the offsets.

What you can do

The following is what you can do depending on the use case.

Archive logs purged too early

Work with DBAs to increase the retention window or configure a secondary archive destination with longer retention. Ensure Debezium has access to both online and archive logs from all redo threads.

Connector is offline for too long

Ask DBAs to restore the missing archive logs if possible. Once restored, Debezium should resume without changes. If restoration isn’t possible:

  • Manually update the connector offsets to a valid SCN that exists in the logs using the procedure in the FAQ or the offset editor when using file-based offsets.

  • Alternatively, clear the offset and history topics and restart the connector. You may choose to take a new full snapshot or use incremental snapshots.

Disproportionate table activity

Configure a heartbeat using heartbeat.interval.ms and heartbeat.action.query to ensure regular SCN progression even if no captured changes occur. Be sure the table used in the heartbeat query is included in the connector’s include list.

Redo thread consistency

Starting in Debezium 2.7, a more sophisticated consistency check was introduced to ensure log mining across all redo threads is reliable. This process is called Redo Thread Consistency.

If you’ve seen errors like the following, you’ve encountered this check in action.

Redo Thread 1 is inconsistent; an archive log with sequence 12345 is not available

What Debezium checks

Before mining starts, Debezium queries the available logs and validates the following:

  • At least one thread has a log containing the required SCN.

  • For each thread, log sequences must be continuous (no gaps), up to the latest sequence number in V$THREAD.

  • All redo threads must pass this check.

  • Threads in OPEN state (typically active instances) are held to stricter standards than those in CLOSED state.

If any redo thread fails this consistency check, Debezium pauses briefly, re-queries the logs, and retries. After 5 failed attempts (by default), it aborts with an SCN not found error.

How to troubleshoot

Start by reviewing the INFO level logs, that’s where Debezium reports which threads or sequences are inconsistent. If you’re not seeing this, increase logging for the following class:

io.debezium.connector.oracle.logminer.LogFileCollector

If your database is under heavy load or experiencing high log churn, the default retry strategy may not be sufficient. You can tune the retry behavior with these settings:

{
  "internal.log.mining.log.query.max.retries": "10",
  "internal.log.mining.log.backoff.initial.delay.ms": "1000",
  "internal.log.mining.log.backoff.max.delay.ms": "60000"
}

These options implement an exponential backoff: 1s, 2s, 4s, …​ until either the logs become consistent, or the max retries are exhausted.

Wrapping up

The error "Online REDO LOG files or archive log files do not contain the offset SCN" can be frustrating, but it’s usually a symptom of deeper log availability or coordination issues. By understanding:

  • How SCNs relate to redo logs and LogMiner

  • The differences between online and archive logs

  • Debezium’s internal consistency checks

you can more effectively diagnose and prevent these issues.

When in doubt check:

  • Offset SCN validity during startup

  • Log retention and thread coverage

  • Redo thread consistency at mining time

  • Heartbeat configurations to avoid staleness in idle streams.

With the right monitoring and configuration, these errors can become rare events rather than recurring blockers.

Chris Cranford

Chris is a software engineer at Red Hat. He previously was a member of the Hibernate ORM team and now works on Debezium. He lives in North Carolina just a few hours from Red Hat towers.

   


About Debezium

Debezium is an open source distributed platform that turns your existing databases into event streams, so applications can see and respond almost instantly to each committed row-level change in the databases. Debezium is built on top of Kafka and provides Kafka Connect compatible connectors that monitor specific database management systems. Debezium records the history of data changes in Kafka logs, so your application can be stopped and restarted at any time and can easily consume all of the events it missed while it was not running, ensuring that all events are processed correctly and completely. Debezium is open source under the Apache License, Version 2.0.

Get involved

We hope you find Debezium interesting and useful, and want to give it a try. Follow us on Twitter @debezium, chat with us on Zulip, or join our mailing list to talk with the community. All of the code is open source on GitHub, so build the code locally and help us improve ours existing connectors and add even more connectors. If you find problems or have ideas how we can improve Debezium, please let us know or log an issue.

×