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

Deadlock during snapshot with Mongo connector

XMLWordPrintable

    • False
    • None
    • False

      In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      MongoDB, 1.9.3.Final but also replicated on 1.8.x

      What is the connector configuration?

      Roughly:
      "connector.class": "io.debezium.connector.mongodb.MongoDbConnector",
      "capture.mode": "oplog",
      "tombstones.on.delete": "false",
      "snapshot.delay.ms": "10000",
      "snapshot.max.threads": "10",
      "mongodb.socket.timeout.ms": "600000",
      "max.queue.size": "32768",
      "max.queue.size.in.bytes": "67108864",
      Running embedded with custom ChangeConsumer and OffsetBackingStore

      What is the captured database version and mode of deployment?

      On Prem 4.2

      What behaviour do you expect?

      No deadlocking

      What behaviour do you see?

      Snapshot runs for a few minutes before "deadlocking".

       

      "debezium-mongodbconnector-prod-profile-oplog-replicator-profile012-snapshot-profile012-8" #75 prio=5 os_prio=0 tid=0x00007f03a4016000 nid=0x6a waiting for monitor entry [0x00007f0498b95000]
         java.lang.Thread.State: BLOCKED (on object monitor)
          at io.debezium.connector.base.ChangeEventQueue.doEnqueue(ChangeEventQueue.java:199)
          - waiting to lock <0x00000006a12673e8> (a io.debezium.connector.base.ChangeEventQueue)
          at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:169)
          at io.debezium.pipeline.EventDispatcher$BufferingSnapshotChangeRecordReceiver.changeRecord(EventDispatcher.java:440)
          at io.debezium.pipeline.EventDispatcher$1.changeRecord(EventDispatcher.java:166)
          at io.debezium.connector.mongodb.MongoDbChangeSnapshotOplogRecordEmitter.emitReadRecord(MongoDbChangeSnapshotOplogRecordEmitter.java:73)
       
      "debezium-mongodbconnector-prod-profile-oplog-replicator-profile012-snapshot-profile012-6" #73 prio=5 os_prio=0 tid=0x00007f03a4012000 nid=0x68 in Object.wait() [0x00007f0498d95000]
         java.lang.Thread.State: TIMED_WAITING (on object monitor)
          at java.lang.Object.wait(Native Method)
          at io.debezium.connector.base.ChangeEventQueue.doEnqueue(ChangeEventQueue.java:204)
          - locked <0x00000006a12673e8> (a io.debezium.connector.base.ChangeEventQueue)
          at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:169)
          at io.debezium.pipeline.EventDispatcher$BufferingSnapshotChangeRecordReceiver.changeRecord(EventDispatcher.java:440)
          at io.debezium.pipeline.EventDispatcher$1.changeRecord(EventDispatcher.java:166)
          at io.debezium.connector.mongodb.MongoDbChangeSnapshotOplogRecordEmitter.emitReadRecord(MongoDbChangeSnapshotOplogRecordEmitter.java:73)
       
      "pool-7-thread-1" #37 prio=5 os_prio=0 tid=0x00007f04cd405000 nid=0x45 waiting for monitor entry [0x00007f04b13fe000]    java.lang.Thread.State: BLOCKED (on object monitor)     at io.debezium.connector.base.ChangeEventQueue.poll(ChangeEventQueue.java:237)     - waiting to lock <0x00000006a12673e8> (a io.debezium.connector.base.ChangeEventQueue)     at io.debezium.connector.mongodb.MongoDbConnectorTask.doPoll(MongoDbConnectorTask.java:171)
      

      I believe it's probably somewhat dependant on the thread scheduler, but when one of the writing threads decides that the queue is full then in a loop it:
      1. notifies a waiting thread.(ie the consuming thread)
      2. waits for a notify (ie for the reading thread to signal that its done).

      With more than one write thread these notifies simply wake each other up while staving the read thread, in our system the read thread just stays blocked trying to enter the synchronised section while the write threads use up ~ 1cpu waking each other up as fast as they can

       

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

      On 1.9.3.final yes

      Do you have the connector logs, ideally from start till finish?

      I can get but there's nothing interesting in them

      How to reproduce the issue using our tutorial deployment?

      No, but I'll try write some java code to replicate the issue

            jpechane Jiri Pechanec
            tim-patterson Tim Patterson (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

              Created:
              Updated:
              Resolved: