Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-36858

[pipeline-kafka] JsonRowDataSerializationSchema compatibility bug

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • cdc-3.2.1
    • None
    • Flink CDC

    Description

      I encountered this bug when I was working on MysqlToKafkaE2eITCase
      minimum repetition step:
      Flink 1.20
      Flink cdc master

      source:
        type: mysql
        name: MySQL Source
        hostname: localhost
        port: 3306
        username: test
        password: test
        tables: testDB.testTable
        server-id: 5206
        server-time-zone: Asia/Shanghai
        schema-change.enabled: true
        
      sink:
        type: kafka
        name: Kafka Sink
        properties.bootstrap.servers: localhost:9092
        topic: test_topic
      pipeline:
        name: MySQL to Kafka Pipeline
        parallelism: 1 

      error message:

      Caused by: java.lang.NoSuchMethodError: 'void org.apache.flink.formats.json.JsonRowDataSerializationSchema.<init>(org.apache.flink.table.types.logical.RowType, org.apache.flink.formats.common.TimestampFormat, org.apache.flink.formats.json.JsonFormatOptions$MapNullKeyMode, java.lang.String, boolean)'
              at org.apache.flink.cdc.connectors.kafka.serialization.JsonSerializationSchema.buildSerializationForPrimaryKey(JsonSerializationSchema.java:134)
              at org.apache.flink.cdc.connectors.kafka.serialization.JsonSerializationSchema.serialize(JsonSerializationSchema.java:101)
              at org.apache.flink.cdc.connectors.kafka.serialization.JsonSerializationSchema.serialize(JsonSerializationSchema.java:47)
              at org.apache.flink.cdc.connectors.kafka.sink.PipelineKafkaRecordSerializationSchema.serialize(PipelineKafkaRecordSerializationSchema.java:99)
              at org.apache.flink.cdc.connectors.kafka.sink.PipelineKafkaRecordSerializationSchema.serialize(PipelineKafkaRecordSerializationSchema.java:44) 

      Cause of error:
      The JsonRowDataSerializationSchema constructor of Flink 1.20 has an additional ignoreNullFields parameter, which causes this part of the code to throw a NoSuchMethodError

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              MOBIN MOBIN
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated: