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

MongoDB ExtractNewDocumentState SMT blocks heartbeat messages

      Heartbeat AVRO messages are blocked by the ExtractNewDocumentState SMT in this point of the code:

      if (record.keySchema() == null ||
                      record.keySchema().name() == null ||
                      !record.keySchema().name().endsWith(RECORD_ENVELOPE_KEY_SCHEMA_NAME_SUFFIX)) {
                  LOGGER.debug("Message without Debezium CDC Envelope ignored, missing Key Schema. Message key: \"{}\"", record.key());
                  return null;
              }
      

      The schema of the heartbeat is:

      {
        "type": "record",
        "name": "ServerNameKeyKey",
        "namespace": "io.debezium.connector.common",
        "fields": [
          {
            "name": "serverName",
            "type": "string"
          }
        ],
        "connect.name": "io.debezium.connector.common.ServerNameKeyKey"
      }
      

      As RECORD_ENVELOPE_KEY_SCHEMA_NAME_SUFFIX is ".Key" and the schema name is instead "ServerNameKeyKey" the message is rejected.

            [DBZ-1513] MongoDB ExtractNewDocumentState SMT blocks heartbeat messages

            Released

            Jiri Pechanec added a comment - Released

            For the change notes:

            The ExtractNewDocumentState and EventRouter SMTs now propagate any heartbeat or schema change messages unchanged instead of dropping them as before. This is to ensure consistency with the ExtractNewRecordState SMT.

            Gunnar Morling added a comment - For the change notes: The ExtractNewDocumentState and EventRouter SMTs now propagate any heartbeat or schema change messages unchanged instead of dropping them as before. This is to ensure consistency with the ExtractNewRecordState SMT.

            I think your PR does the right thing – pass on any message that's not of interest to the SMT unchanged, i.e. consistently use the behavior of ExtractNewRecordState. Which is good also in terms of compatability, as that SMT is the one used the most by far.

            Gunnar Morling added a comment - I think your PR does the right thing – pass on any message that's not of interest to the SMT unchanged, i.e. consistently use the behavior of ExtractNewRecordState . Which is good also in terms of compatability, as that SMT is the one used the most by far.

            gunnar.morling renatomefi We should make a decision for what is a proper handling of messages in our SMTs.

            For rehresh, right now we generate three types of messages

            • events
            • heartbeats
            • schema change info messages

            In ExtractNewRecordState the messages are passed without any changes, in EventRouter and ExtractNewDocumentState they are filtered out.

            What should be the correct behaviour? IMHO the passing downstream is the correct one as we don't mix events of multiple types in a single topic. This works correctly for both source and sink. The only problem I can see is when the SMT is applied on the sink side and multiple event types are mixed together in the same topic. Then it would make sense to do the filtering.

            So IMHO we should either use the pass to downstream semantic in all SMTs or if we want to handle special cases we should intorduce a config parameter that will have the following options

            • PASS - send downstream
            • IGNORE - filter out
            • PASS_AND_WARN
            • IGNORE_AND_WARN
            • ERROR

            Jiri Pechanec added a comment - gunnar.morling renatomefi We should make a decision for what is a proper handling of messages in our SMTs. For rehresh, right now we generate three types of messages events heartbeats schema change info messages In ExtractNewRecordState the messages are passed without any changes, in EventRouter and ExtractNewDocumentState they are filtered out. What should be the correct behaviour? IMHO the passing downstream is the correct one as we don't mix events of multiple types in a single topic. This works correctly for both source and sink. The only problem I can see is when the SMT is applied on the sink side and multiple event types are mixed together in the same topic. Then it would make sense to do the filtering. So IMHO we should either use the pass to downstream semantic in all SMTs or if we want to handle special cases we should intorduce a config parameter that will have the following options PASS - send downstream IGNORE - filter out PASS_AND_WARN IGNORE_AND_WARN ERROR

              jpechane Jiri Pechanec
              cosimomeli92@hotmail.it Cosimo Meli (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: