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

JdbcSchemaHistory Fails to Handle Data Sharding When Recovering Records

XMLWordPrintable

    • Important

      The JdbcSchemaHistory component writes schema data using data sharding (splitting large data into fragments). However, during recovery in the recoverRecords method, it directly reads the history_data field from a single database record via

      try (
          Statement stmt = conn.createStatement();
          ResultSet rs = stmt.executeQuery(this.config.getTableSelect());
      ) {
          while(rs.next()) {
              String historyData = rs.getString("history_data");
              if (!historyData.isEmpty()) {
                  try {
                      records.accept(new HistoryRecord(this.reader.read(historyData)));
                  } catch (IOException e) {
                      throw new DebeziumException(e);
                  }
              }
          }
      } 

       

      This approach fails to handle sharded data (e.g., when large schema changes are split across multiple records). When monitoring SQL Server with large table structures, this causes exceptions due to incomplete or truncated data during recovery.

      Bug report

       

      What Debezium connector do you use and what version?

      3.0.8 - 3.1.1.Final

      What is the connector configuration?

      props.setProperty("offset.storage", JdbcOffsetBackingStore.class.getCanonicalName());
      props.setProperty("offset.storage.jdbc.url", this.jdbcUrl);
      props.setProperty("offset.storage.jdbc.user", this.jdbcUser);
      props.setProperty("offset.storage.jdbc.password", this.jdbcPassword);

      final String DEFAULT_TABLE_DDL = "CREATE TABLE %s (id VARCHAR(36) NOT NULL, offset_key VARCHAR(4096), offset_val VARCHAR(4096),record_insert_ts TIMESTAMP NOT NULL,record_insert_seq INTEGER NOT NULL)";
      props.setProperty("offset.storage.jdbc.table.ddl", DEFAULT_TABLE_DDL);
      props.setProperty("offset.storage.jdbc.table.name", this.jdbcOffsetTableName);
      props.setProperty("schema.history.internal", JdbcSchemaHistory.class.getCanonicalName());
      props.setProperty("schema.history.internal.jdbc.url", this.jdbcUrl);
      props.setProperty("schema.history.internal.jdbc.user", this.jdbcUser);
      props.setProperty("schema.history.internal.jdbc.password", this.jdbcPassword);
      final String DEFAULT_HISTORY_TABLE_DDL = "CREATE TABLE %s (id VARCHAR(255) NOT NULL, history_data LONGTEXT, history_data_seq INTEGER, record_insert_ts TIMESTAMP NOT NULL, record_insert_seq INTEGER NOT NULL)";
      props.setProperty("schema.history.internal.jdbc.table.ddl", DEFAULT_HISTORY_TABLE_DDL);
      props.setProperty("schema.history.internal.jdbc.table.name", this.jdbcSchemaTableName);
      final String DEFAULT_TABLE_SELECT = "SELECT id, history_data, history_data_seq FROM %s ORDER BY record_insert_ts, record_insert_seq";
      props.setProperty("schema.history.internal.jdbc.table.select", DEFAULT_TABLE_SELECT);

      What is the captured database version and mode of deployment?

      sqlserver 15.00.4355

      What behavior do you expect?

      recover success

      Do you see the same behaviour using the latest released Debezium version?

      yes.

              Unassigned Unassigned
              tmq777 T MQ (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

                Created:
                Updated:
                Resolved: