Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-2982

Debezium Oracle Connector will appear stuck on large SCN jumps

    XMLWordPrintable

Details

    • False
    • False
    • Undefined

    Description

      Right now the current approach for walking through Logminer entries is to use a resizable window algorithm for executing a bounded SELECT query that fetches all content changes from a start SCN to an end SCN. The algorithm is "resizable" because the end SCN is calculated based on a starting batch size (20000) that gets increased or decreased depending on whether we are lagging from the current SCN or not. In the current codebase the maximum size that this window can take is 100000 units. The increase step used right now is 1000 units.

      This probably works well in many scenarios. However in our current database with several RAC nodes and a hot-standby setup syncronised by Shareplex it has proven problematic. The reason is that we tend to see large to very large SCN jumps ( e.g. I've seen jumps ranging 200K to 5000M units ). The consequence of those large jumps is that as the start SCN is very far away from the end SCN, Debezium needs to catch up. Then as as both the maximum batch size and the batch increase steps are very small compared to those huge jumps, it takes up to several hours for Debezium to start producing results again. It is important to note that Debezium is working fine. It is just that spends hours pulling no results.

      Digging into why those SCN jumps happen is still ongoing in our side. I've found that database links are a common cause for an SCN jump. Arguably large reports or large batch operations could also create jumps too. But I have the feeling that in our particular case, Shareplex could be a factor there.

      Nevertheless I believe that two approaches can be taken here:

      a) Would be to make some of the parameters customizable. Like the default batch size and steps, so teams needing a different behavior can adapt the connector to their needs.
      b) Would be to change the querying algorithm to use a start SCN and a fetch rows clause. The limit for the fetch rows statement would be the current fetch limit. This essentially ends up being something like this in terms of SQL:

      return "SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME " +
          " FROM " + LOGMNR_CONTENTS_VIEW + " WHERE  OPERATION_CODE in (1,2,3,5) " + // 5 - DDL
          " AND SEG_OWNER = '" + schemaName.toUpperCase() + "' " +
          buildTableInPredicate(whiteListTableNames) +
          " AND SCN >= ? " +
           // Capture DDL and MISSING_SCN rows only hwne not performed by SYS, SYSTEM, and LogMiner user
          " OR (OPERATION_CODE IN (5,34) AND USERNAME NOT IN ('SYS','SYSTEM','" + logMinerUser.toUpperCase() + "')) " +
          // Capture all COMMIT and ROLLBACK operations performed by the database
          " OR (OPERATION_CODE IN (7,36)) " +
          sorting + // todo username = schemaName?
          " FETCH NEXT ? ROWS ONLY";
      

      c) Create some other sort of adapting batch sizing algorithm that can work both with small and very large increments and can react to large changes.

      For what is worth, we have been successfully running a connector fork that uses approach b) and can provide a PR if the team thinks that this is a good approach.

      Attachments

        Activity

          People

            ccranfor@redhat.com Chris Cranford
            martper2 Martin Perez (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: