• Icon: Feature Request Feature Request
    • Resolution: Done
    • Icon: Major Major
    • 1.0.0.CR1
    • 0.9.3.Final
    • mongodb-connector
    • None

      MongoDB connector did not get any change event from MongoDB when I insert multiple documents in transaction. Without transaction it works fine. Does MongoDB connector support transaction?

            [DBZ-1215] Support MongoDB 4.0 transaction

            Released

            Jiri Pechanec added a comment - Released

            noam.garber I've repuporsed this one for it

            Jiri Pechanec added a comment - noam.garber I've repuporsed this one for it

            Should I open a "feature request", for tracking this?

            Thanks,
            Noam

            NOAM GARBER (Inactive) added a comment - Should I open a "feature request", for tracking this? Thanks, Noam

            Thansk jpechane for the quick reply.
            Is there a place in your documentation you mention this? I read a few times, and looked for some information, and could not find any.
            I am asking, so I would track this and see once this is resolved.

            Thanks,
            Noam

            NOAM GARBER (Inactive) added a comment - Thansk jpechane for the quick reply. Is there a place in your documentation you mention this? I read a few times, and looked for some information, and could not find any. I am asking, so I would track this and see once this is resolved. Thanks, Noam

            noam.garber Thats for the information. We don't officially support transactions yet.

            Jiri Pechanec added a comment - noam.garber Thats for the information. We don't officially support transactions yet.

            NOAM GARBER (Inactive) added a comment - - edited

            Hi,
            I encountered this issue also.
            The reason the transactions are not caught is in your code.
            This is an example of a transaction, as appearing in the oplog.

            [2019-11-07 17:34:52.174 +0200] [DEBUG] [debezium-mongodbconnector-my-mongo-db-replicator-0] [Replicator] []-  Found event: Document{{ts=Timestamp{value=6756588683140268033, seconds=1573140892, inc=1}, t=133, h=0, v=2, {color:#DE350B}op=c,{color} ns=admin.$cmd, wall=Thu Nov 07 17:34:52 IST 2019, lsid=Document{{id=5809a590-f302-4b6c-9aec-30d0c4fcaa24, uid=org.bson.types.Binary@b2e9fd42}}, txnNumber=0, prevOpTime=Document{{ts=Timestamp{value=0, seconds=0, inc=0}, t=-1}}, o=Document{{applyOps=[Document{{op=i, ns=noamDb.noamCollection, ui=8aeb4e62-7488-466a-9335-de318172c95c, o=Document{{_id=5dc439929eb72c8102587677, aaaa=444.0}}}}, Document{{op=i, ns=noamDb.noamCollection, ui=8aeb4e62-7488-466a-9335-de318172c95c, o=Document{{_id=5dc439979eb72c8102587678, aaaa=7777.0}}}}]}}}} 
            

            in your code(Replicator.java, handleOplogEvent), it seems you are filtering our operations which are not insert/update/delete

                   String operation = event.getString("op");
                    // the op is not insert/update/delete
                    if (!RecordMakers.isValidOperation(operation)) {
                        LOGGER.debug("Skipping event with \"op={}\"", operation);
                        return true;
                    }
            

            And these are the valid operations:

                    literals.put("i", Operation.CREATE);
                    literals.put("u", Operation.UPDATE);
                    literals.put("d", Operation.DELETE);
            

            In the oplog example, you can see the transaction operation is "c" and does not match any of the cases
            Questions-
            Is there a way to configure Mongo not to write* c *as the op and force Debezium to catch the operation in the oplog?
            Is there a way to force debezium to catch the operation and handle the event.
            Do you officially support Transactions?

            Thanks,
            Noam

            NOAM GARBER (Inactive) added a comment - - edited Hi, I encountered this issue also. The reason the transactions are not caught is in your code. This is an example of a transaction, as appearing in the oplog. [2019-11-07 17:34:52.174 +0200] [DEBUG] [debezium-mongodbconnector-my-mongo-db-replicator-0] [Replicator] []- Found event: Document{{ts=Timestamp{value=6756588683140268033, seconds=1573140892, inc=1}, t=133, h=0, v=2, {color:#DE350B}op=c,{color} ns=admin.$cmd, wall=Thu Nov 07 17:34:52 IST 2019, lsid=Document{{id=5809a590-f302-4b6c-9aec-30d0c4fcaa24, uid=org.bson.types.Binary@b2e9fd42}}, txnNumber=0, prevOpTime=Document{{ts=Timestamp{value=0, seconds=0, inc=0}, t=-1}}, o=Document{{applyOps=[Document{{op=i, ns=noamDb.noamCollection, ui=8aeb4e62-7488-466a-9335-de318172c95c, o=Document{{_id=5dc439929eb72c8102587677, aaaa=444.0}}}}, Document{{op=i, ns=noamDb.noamCollection, ui=8aeb4e62-7488-466a-9335-de318172c95c, o=Document{{_id=5dc439979eb72c8102587678, aaaa=7777.0}}}}]}}}} in your code(Replicator.java, handleOplogEvent), it seems you are filtering our operations which are not insert/update/delete String operation = event.getString("op"); // the op is not insert/update/delete if (!RecordMakers.isValidOperation(operation)) { LOGGER.debug("Skipping event with \"op={}\"", operation); return true; } And these are the valid operations: literals.put("i", Operation.CREATE); literals.put("u", Operation.UPDATE); literals.put("d", Operation.DELETE); In the oplog example, you can see the transaction operation is "c" and does not match any of the cases Questions- Is there a way to configure Mongo not to write* c *as the op and force Debezium to catch the operation in the oplog? Is there a way to force debezium to catch the operation and handle the event. Do you officially support Transactions? Thanks, Noam

            Hi, could you please collect logs with `io.debezium.connector.mongodb.Replicator` set to `DEBUG` so we see the events that are coming from your oplog?

            Thanks

            Jiri Pechanec added a comment - Hi, could you please collect logs with `io.debezium.connector.mongodb.Replicator` set to `DEBUG` so we see the events that are coming from your oplog? Thanks

              jpechane Jiri Pechanec
              shilva Worawat Wijarn (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: