Bug reporter information
Full name
Jan Bejgier
Email or GitHub username
jan.bejgier@goldenore.com
Company or organization
Goldenore Sp. z o.o.
Bug description
When OLR is restarted during data transmission to the Debezium connector, one message is lost.
OLR, after receiving a CONTINUE request with a given c_scn and c_idx, resumes streaming data but always skips message that is immediately following the requested c_idx.
Step-by-step reproduction instructions
-
Clone the required repository:
git clone https://github.com/bersler/OpenLogReplicator-tutorials.git
-
Create the required images:
cd OpenLogReplicator-tutorials/images/
./oracle_database_21.3.0-xe.sh
./bersler_openlogreplicator_tutorial_dev.sh
-
Start the Oracle database with modified compose.yaml:
services:
oracle:
image: "${DB_IMAGE}"
container_name: ORA1
volumes:
- ./fra:/opt/oracle/fra
- ./oradata:/opt/oracle/oradata
- ./sql:/opt/sql
- ./setup:/opt/oracle/scripts/setup
networks:
- isolated_network
environment:
- ORACLE_CHARACTERSET=AL32UTF8
- ORACLE_PWD=oracle
build:
context: .
shm_size: '2gb'
shm_size: '2gb'
ports:
- "1521:1521"
kafka:
image: "${KAFKA_IMAGE}"
container_name: KAFKA1
profiles: ["kafka"]
volumes:
- ./kafka:/var/lib/kafka/data
networks:
- isolated_network
ports:
- "9092:9092"
environment:
# KRaft parameters
- CLUSTER_ID=${KAFKA_CLUSTER}
- KAFKA_NODE_ID=1
- KAFKA_PROCESS_ROLES=broker,controller
- KAFKA_CONTROLLER_QUORUM_BOOTSTRAP_SERVERS=kafka:9093
# Listeners
- KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093
- KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9092
- KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER
- KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
# Performance & configuration
- KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS=0
- KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=1
- KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1
- KAFKA_TRANSACTION_STATE_LOG_MIN_ISR=1
- KAFKA_AUTO_CREATE_TOPICS_ENABLE=true
healthcheck:
test: ["CMD", "/kafka/bin/kafka-topics.sh", "--bootstrap-server", "${KAFKA_BROKER}", "--list"]
interval: 10s
timeout: 5s
retries: 5
start_period: 60s
openlogreplicator:
image: "${OLR_IMAGE}"
container_name: "${OLR_CONTAINER}"
profiles: ["openlogreplicator"]
depends_on:
- oracle
- kafka
volumes:
- ./bin:/opt/bin
- ./checkpoint:/opt/OpenLogReplicator/checkpoint
- ./fra:/opt/fra
- ./log:/opt/OpenLogReplicator/log
- ./oradata:/opt/oradata
- ./scripts:/opt/OpenLogReplicator/scripts
restart: "no"
ports:
- "50000:50000"
networks:
- isolated_network
kafka-connect:
image: quay.io/debezium/connect:latest
container_name: KAFKA-CONNECT
profiles: ["connect"]
environment:
BOOTSTRAP_SERVERS: kafka:9092
GROUP_ID: 1
CONFIG_STORAGE_TOPIC: connect_configs
OFFSET_STORAGE_TOPIC: connect_offsets
STATUS_STORAGE_TOPIC: connect_statuses
ports:
- "8083:8083"
depends_on:
- oracle
- kafka
networks:
- isolated_network
networks:
isolated_network:
cd ../tutorials/Oracle-to-Kafka/
./2.db-start.sh
-
Create directory kafka:
mkdir kafka
chmod 777 kafka
- Add line to
sql/schema-usrolr.sql:
GRANT SELECT ON SYS.V_$PDBS TO USROLR;
-
Configure OpenLogReplicator:
{
"version": "1.9.0",
"log-level": 4,
"source": [
{
"alias": "S1",
"name": "ORA1",
"reader": {
"type": "online",
"path-mapping": ["/opt/oracle/oradata", "/opt/oradata", "/opt/oracle/fra", "/opt/fra"],
"user": "USROLR",
"password": "USROLRPWD",
"server": "//oracle:1521/XEPDB1"
},
"format": {
"type": "json",
"db": 3,
"interval-dts": 9,
"interval-ytm": 4,
"message": 2,
"rid": 1,
"column": 2,
"schema": 7,
"scn-type": 5,
"timestamp-type": 4,
"flush-buffer": 0
},
"filter": {
"table": [
{"owner": "USRTBL", "table": "TAB1.*"}
]
}
}
],
"target": [
{
"alias": "T1",
"source": "S1",
"writer": {
"type": "network",
"uri": "0.0.0.0:50000",
"queue-size": 1000000
}
}
]
}
-
Modify file and run file 4.olr-start.sh:
set -e
. cfg.sh
. ../common/functions.sh
echo "4. creating and starting OpenLogReplicator container"
olr_files
db_sql "${DB_CONTAINER}" /opt/sql/schema-usrolr.sql /opt/sql/schema-usrolr.out
docker_up --profile openlogreplicator --profile kafka --profile connect
finish
-
Create debezium connector:
curl -X POST http://localhost:8083/connectors/ \
-H "Content-Type: application/json" \
-d '{
"name": "oracle-olr-connector",
"config": {
"connector.class": "io.debezium.connector.oracle.OracleConnector",
"database.hostname": "oracle",
"database.port": "1521",
"database.user": "sys as sysdba",
"database.password": "oracle",
"database.dbname": "XEPDB1",
"database.connection.adapter": "olr",
"openlogreplicator.host": "openlogreplicator",
"openlogreplicator.port": "50000",
"openlogreplicator.source": "ORA1",
"topic.prefix": "oracle_olr",
"snapshot.mode": "no_data",
"schema.history.internal.kafka.bootstrap.servers": "kafka:9092",
"schema.history.internal.kafka.topic": "schema-changes.oracle",
"table.include.list": "USRTBL.TAB1",
"database.history.store.only.captured.tables.ddl": "true"
}
}'
-
Start:
CREATE TABLE USRTBL.TAB1(
ID NUMBER NOT NULL,
COUNT NUMBER
);
ALTER TABLE USRTBL.TAB1 ADD CONSTRAINT ID_PK PRIMARY KEY(ID);
ALTER TABLE USRTBL.TAB1 ADD SUPPLEMENTAL LOG DATA (PRIMARY KEY) COLUMNS;
INSERT INTO USRTBL.TAB1 (ID, COUNT)
SELECT
LEVEL - 1 AS ID,
MOD(LEVEL, 100) AS COUNT
FROM DUAL
CONNECT BY LEVEL <= 500000;
COMMIT;
-
Restart OLR docker container during transaction:
Restarting OLR will create Connection lost error in connector, so it need to be restarted too:
curl -X PUT http://localhost:8083/connectors/oracle-olr-connector/stop
curl -X PUT http://localhost:8083/connectors/oracle-olr-connector/resume
- Observe results:
docker exec -it KAFKA1 /kafka/bin/kafka-get-offsets.sh \
--bootstrap-server localhost:9092 \
--topic oracle_olr.USRTBL.TAB1 \
--time -1
Actual result:
oracle_olr.USRTBL.ADAM1:0:499999
Expected result:
oracle_olr.USRTBL.ADAM1:0:500000
OLR logs with STREAM trace snippet after restart:
2026-04-21 11:02:23 INFO 00000 new client connection: 172.19.0.5:39222
2026-04-21 11:02:23 TRACE STRM request: INFO: ORA1
2026-04-21 11:02:23 TRACE STRM request: CONTINUE database: ORA1, c_scn: 2629475, c_idx: 28763
2026-04-21 11:02:23 INFO 00000 client requested scn: 2629475, idx: 28763
2026-04-21 11:02:23 INFO 00000 streaming to client
2026-04-21 11:02:24 TRACE STRM data[421]: [{"scn":2631626,"tm":1776769311000000000,"c_scn":2629475,"c_idx":28765,"xid":"0x000a.01a.00000268","db":"XEPDB1","payload":[{"op":"c","schema":{"owner":"USRTBL","table":"TAB1","obj":75754,"columns":[{"name":"ID","type":"number","precision":-1,"scale":-1,"nullable":false},{"name":"COUNT","type":"number","precision":-1,"scale":-1,"nullable":true}]},"num":28763,"rid":"AAASftAANAAAAEEAAe","after":{"ID":28763,"COUNT":64}}]}]
In this case, OLR is expected to start streaming data from c_idx = 28764, but it starts from 28765 instead.
Proposed Fix
I noticed that in Builder.h, in the builderBegin() method, the parameter msg->lwnIdx is assigned before the lwnIdx variable is incremented:
I tested changing it to:
After applying this change, the described issue no longer occurs.
I ran multiple test transactions, including long-running workloads, and did not observe any data loss. All messages were delivered correctly.
Environment details (required)
- Official binary signature/version:
v1.9.0
- OpenLogReplicator version:
1.9.0
- Oracle version and edition:
Oracle Database 21c Express Edition Release 21.0.0.0.0 - Production / Oracle Database 19c Standard Edition 2 Release 19.0.0.0.0 - Production
- Operating system:
Debian 13.0 / Red Hat 9.7
- Docker version:
29.2.1
Bug reporter information
Full name
Jan BejgierEmail or GitHub username
jan.bejgier@goldenore.comCompany or organization
Goldenore Sp. z o.o.Bug description
When OLR is restarted during data transmission to the Debezium connector, one message is lost.
OLR, after receiving a
CONTINUErequest with a givenc_scnandc_idx, resumes streaming data but always skips message that is immediately following the requestedc_idx.Step-by-step reproduction instructions
Clone the required repository:
Create the required images:
cd OpenLogReplicator-tutorials/images/ ./oracle_database_21.3.0-xe.sh ./bersler_openlogreplicator_tutorial_dev.shStart the Oracle database with modified
compose.yaml:cd ../tutorials/Oracle-to-Kafka/ ./2.db-start.shCreate directory
kafka:sql/schema-usrolr.sql:Configure OpenLogReplicator:
{ "version": "1.9.0", "log-level": 4, "source": [ { "alias": "S1", "name": "ORA1", "reader": { "type": "online", "path-mapping": ["/opt/oracle/oradata", "/opt/oradata", "/opt/oracle/fra", "/opt/fra"], "user": "USROLR", "password": "USROLRPWD", "server": "//oracle:1521/XEPDB1" }, "format": { "type": "json", "db": 3, "interval-dts": 9, "interval-ytm": 4, "message": 2, "rid": 1, "column": 2, "schema": 7, "scn-type": 5, "timestamp-type": 4, "flush-buffer": 0 }, "filter": { "table": [ {"owner": "USRTBL", "table": "TAB1.*"} ] } } ], "target": [ { "alias": "T1", "source": "S1", "writer": { "type": "network", "uri": "0.0.0.0:50000", "queue-size": 1000000 } } ] }Modify file and run file
4.olr-start.sh:Create debezium connector:
Start:
Restart OLR docker container during transaction:
Restarting OLR will create
Connection losterror in connector, so it need to be restarted too:docker exec -it KAFKA1 /kafka/bin/kafka-get-offsets.sh \ --bootstrap-server localhost:9092 \ --topic oracle_olr.USRTBL.TAB1 \ --time -1Actual result:
Expected result:
OLR logs with
STREAMtrace snippet after restart:2026-04-21 11:02:23 INFO 00000 new client connection: 172.19.0.5:39222 2026-04-21 11:02:23 TRACE STRM request: INFO: ORA1 2026-04-21 11:02:23 TRACE STRM request: CONTINUE database: ORA1, c_scn: 2629475, c_idx: 28763 2026-04-21 11:02:23 INFO 00000 client requested scn: 2629475, idx: 28763 2026-04-21 11:02:23 INFO 00000 streaming to client 2026-04-21 11:02:24 TRACE STRM data[421]: [{"scn":2631626,"tm":1776769311000000000,"c_scn":2629475,"c_idx":28765,"xid":"0x000a.01a.00000268","db":"XEPDB1","payload":[{"op":"c","schema":{"owner":"USRTBL","table":"TAB1","obj":75754,"columns":[{"name":"ID","type":"number","precision":-1,"scale":-1,"nullable":false},{"name":"COUNT","type":"number","precision":-1,"scale":-1,"nullable":true}]},"num":28763,"rid":"AAASftAANAAAAEEAAe","after":{"ID":28763,"COUNT":64}}]}]In this case, OLR is expected to start streaming data from
c_idx = 28764, but it starts from28765instead.Proposed Fix
I noticed that in
Builder.h, in thebuilderBegin()method, the parametermsg->lwnIdxis assigned before thelwnIdxvariable is incremented:I tested changing it to:
After applying this change, the described issue no longer occurs.
I ran multiple test transactions, including long-running workloads, and did not observe any data loss. All messages were delivered correctly.
Environment details (required)
v1.9.01.9.0Oracle Database 21c Express Edition Release 21.0.0.0.0 - Production/Oracle Database 19c Standard Edition 2 Release 19.0.0.0.0 - ProductionDebian 13.0/Red Hat 9.729.2.1