diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/Buffer.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/Buffer.java index 94a8a074db3..d756d67e7f4 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/Buffer.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/Buffer.java @@ -15,18 +15,18 @@ public interface Buffer { /** - * to add a {@link SinkRecordDescriptor} to the internal buffer and + * to add a {@link JdbcSinkRecord} to the internal buffer and * call the {@link Buffer#flush()} when buffer size >= {@link JdbcSinkConnectorConfig#getBatchSize()} - * @param recordDescriptor the Sink record descriptor + * @param record the Debezium sink record * @return the buffer records */ - List add(SinkRecordDescriptor recordDescriptor); + List add(JdbcSinkRecord record); /** * to clear and flush the internal buffer - * @return {@link SinkRecordDescriptor} the flushed buffer records. + * @return {@link JdbcSinkRecord} the flushed buffer records. */ - List flush(); + List flush(); /** * to check whether buffer is empty or not. diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcChangeEventSink.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcChangeEventSink.java index a964df7551f..49f71856294 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcChangeEventSink.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcChangeEventSink.java @@ -6,6 +6,7 @@ package io.debezium.connector.jdbc; import static io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode.NONE; +import static io.debezium.connector.jdbc.JdbcSinkRecord.FieldDescriptor; import java.sql.SQLException; import java.time.Duration; @@ -27,15 +28,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.debezium.connector.jdbc.SinkRecordDescriptor.FieldDescriptor; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.relational.TableDescriptor; import io.debezium.metadata.CollectionId; +import io.debezium.sink.DebeziumSinkRecord; import io.debezium.sink.spi.ChangeEventSink; import io.debezium.util.Clock; import io.debezium.util.Metronome; import io.debezium.util.Stopwatch; -import io.debezium.util.Strings; /** * A {@link ChangeEventSink} for a JDBC relational database. @@ -48,6 +48,7 @@ public class JdbcChangeEventSink implements ChangeEventSink { public static final String SCHEMA_CHANGE_VALUE = "SchemaChangeValue"; public static final String DETECT_SCHEMA_CHANGE_RECORD_MSG = "Schema change records are not supported by JDBC connector. Adjust `topics` or `topics.regex` to exclude schema change topic."; + private final JdbcSinkConnectorConfig config; private final DatabaseDialect dialect; private final StatelessSession session; @@ -68,13 +69,13 @@ public JdbcChangeEventSink(JdbcSinkConnectorConfig config, StatelessSession sess LOGGER.info("Database version {}.{}.{}", version.getMajor(), version.getMinor(), version.getMicro()); } - @Override public void execute(Collection records) { final Map upsertBufferByTable = new LinkedHashMap<>(); final Map deleteBufferByTable = new LinkedHashMap<>(); - for (SinkRecord record : records) { + for (SinkRecord kafkaSinkRecord : records) { + JdbcSinkRecord record = new JdbcKafkaSinkRecord(kafkaSinkRecord, config.getPrimaryKeyMode(), config.getPrimaryKeyFields(), config.getFieldFilter(), dialect); LOGGER.trace("Processing {}", record); validate(record); @@ -82,25 +83,22 @@ public void execute(Collection records) { Optional optionalCollectionId = getCollectionIdFromRecord(record); if (optionalCollectionId.isEmpty()) { - LOGGER.warn("Ignored to write record from topic '{}' partition '{}' offset '{}'. No resolvable table name", record.topic(), record.kafkaPartition(), - record.kafkaOffset()); + LOGGER.warn("Ignored to write record from topic '{}' partition '{}' offset '{}'. No resolvable table name", record.topicName(), record.partition(), + record.offset()); continue; } - SinkRecordDescriptor sinkRecordDescriptor = buildRecordSinkDescriptor(record); - - if (sinkRecordDescriptor.isTombstone()) { + if (record.isTombstone()) { // Skip only Debezium Envelope tombstone not the one produced by ExtractNewRecordState SMT - LOGGER.debug("Skipping tombstone record {}", sinkRecordDescriptor); + LOGGER.debug("Skipping tombstone record {}", record); continue; } final CollectionId collectionId = optionalCollectionId.get(); - if (sinkRecordDescriptor.isTruncate()) { - + if (record.isTruncate()) { if (!config.isTruncateEnabled()) { - LOGGER.debug("Truncates are not enabled, skipping truncate for topic '{}'", sinkRecordDescriptor.getTopicName()); + LOGGER.debug("Truncates are not enabled, skipping truncate for topic '{}'", record.topicName()); continue; } @@ -109,7 +107,7 @@ public void execute(Collection records) { flushBuffers(deleteBufferByTable); try { - final TableDescriptor table = checkAndApplyTableChangesIfNeeded(collectionId, sinkRecordDescriptor); + final TableDescriptor table = checkAndApplyTableChangesIfNeeded(collectionId, record); writeTruncate(dialect.getTruncateStatement(table)); continue; } @@ -118,10 +116,9 @@ public void execute(Collection records) { } } - if (sinkRecordDescriptor.isDelete()) { - + if (record.isDelete()) { if (!config.isDeleteEnabled()) { - LOGGER.debug("Deletes are not enabled, skipping delete for topic '{}'", sinkRecordDescriptor.getTopicName()); + LOGGER.debug("Deletes are not enabled, skipping delete for topic '{}'", record.topicName()); continue; } @@ -132,10 +129,8 @@ public void execute(Collection records) { flushBufferWithRetries(collectionId, upsertBufferByTable.get(collectionId).flush()); } - Buffer tableIdBuffer = resolveBuffer(deleteBufferByTable, collectionId, sinkRecordDescriptor); - - List toFlush = tableIdBuffer.add(sinkRecordDescriptor); - + Buffer tableIdBuffer = resolveBuffer(deleteBufferByTable, collectionId, record); + List toFlush = tableIdBuffer.add(record); flushBufferWithRetries(collectionId, toFlush); } else { @@ -148,9 +143,9 @@ public void execute(Collection records) { Stopwatch updateBufferStopwatch = Stopwatch.reusable(); updateBufferStopwatch.start(); - Buffer tableIdBuffer = resolveBuffer(upsertBufferByTable, collectionId, sinkRecordDescriptor); + Buffer tableIdBuffer = resolveBuffer(upsertBufferByTable, collectionId, record); - List toFlush = tableIdBuffer.add(sinkRecordDescriptor); + List toFlush = tableIdBuffer.add(record); updateBufferStopwatch.stop(); LOGGER.trace("[PERF] Update buffer execution time {}", updateBufferStopwatch.durations()); @@ -163,22 +158,15 @@ public void execute(Collection records) { flushBuffers(deleteBufferByTable); } - private void validate(SinkRecord record) { - - if (isSchemaChange(record)) { + private void validate(JdbcSinkRecord record) { + if (record.isSchemaChange()) { LOGGER.error(DETECT_SCHEMA_CHANGE_RECORD_MSG); throw new DataException(DETECT_SCHEMA_CHANGE_RECORD_MSG); } } - private static boolean isSchemaChange(SinkRecord record) { - return record.valueSchema() != null - && !Strings.isNullOrEmpty(record.valueSchema().name()) - && record.valueSchema().name().contains(SCHEMA_CHANGE_VALUE); - } - - private Buffer resolveBuffer(Map bufferMap, CollectionId collectionId, SinkRecordDescriptor sinkRecordDescriptor) { - if (config.isUseReductionBuffer() && !sinkRecordDescriptor.getKeyFieldNames().isEmpty()) { + private Buffer resolveBuffer(Map bufferMap, CollectionId collectionId, JdbcSinkRecord record) { + if (config.isUseReductionBuffer() && !record.keyFieldNames().isEmpty()) { return bufferMap.computeIfAbsent(collectionId, k -> new ReducedRecordBuffer(config)); } else { @@ -186,29 +174,11 @@ private Buffer resolveBuffer(Map bufferMap, CollectionId c } } - private SinkRecordDescriptor buildRecordSinkDescriptor(SinkRecord record) { - - SinkRecordDescriptor sinkRecordDescriptor; - try { - sinkRecordDescriptor = SinkRecordDescriptor.builder() - .withPrimaryKeyMode(config.getPrimaryKeyMode()) - .withPrimaryKeyFields(config.getPrimaryKeyFields()) - .withFieldFilter(config.getFieldFilter()) - .withSinkRecord(record) - .withDialect(dialect) - .build(); - } - catch (Exception e) { - throw new ConnectException("Failed to process a sink record", e); - } - return sinkRecordDescriptor; - } - private void flushBuffers(Map bufferByTable) { bufferByTable.forEach((collectionId, recordBuffer) -> flushBufferWithRetries(collectionId, recordBuffer.flush())); } - private void flushBufferWithRetries(CollectionId collectionId, List toFlush) { + private void flushBufferWithRetries(CollectionId collectionId, List toFlush) { int retries = 0; Exception lastException = null; @@ -241,7 +211,7 @@ private void flushBufferWithRetries(CollectionId collectionId, List toFlush) throws SQLException { + private void flushBuffer(CollectionId collectionId, List toFlush) throws SQLException { Stopwatch flushBufferStopwatch = Stopwatch.reusable(); Stopwatch tableChangesStopwatch = Stopwatch.reusable(); if (!toFlush.isEmpty()) { @@ -270,17 +240,17 @@ public void close() { } } - private TableDescriptor checkAndApplyTableChangesIfNeeded(CollectionId collectionId, SinkRecordDescriptor descriptor) throws SQLException { + private TableDescriptor checkAndApplyTableChangesIfNeeded(CollectionId collectionId, JdbcSinkRecord record) throws SQLException { if (!hasTable(collectionId)) { // Table does not exist, lets attempt to create it. try { - return createTable(collectionId, descriptor); + return createTable(collectionId, record); } catch (SQLException ce) { // It's possible the table may have been created in the interim, so try to alter. LOGGER.warn("Table creation failed for '{}', attempting to alter the table", collectionId.toFullIdentiferString(), ce); try { - return alterTableIfNeeded(collectionId, descriptor); + return alterTableIfNeeded(collectionId, record); } catch (SQLException ae) { // The alter failed, hard stop. @@ -292,7 +262,7 @@ private TableDescriptor checkAndApplyTableChangesIfNeeded(CollectionId collectio else { // Table exists, lets attempt to alter it if necessary. try { - return alterTableIfNeeded(collectionId, descriptor); + return alterTableIfNeeded(collectionId, record); } catch (SQLException ae) { LOGGER.error("Failed to alter the table '{}'.", collectionId.toFullIdentiferString(), ae); @@ -309,7 +279,7 @@ private TableDescriptor readTable(CollectionId collectionId) { return session.doReturningWork((connection) -> dialect.readTable(connection, collectionId)); } - private TableDescriptor createTable(CollectionId collectionId, SinkRecordDescriptor record) throws SQLException { + private TableDescriptor createTable(CollectionId collectionId, JdbcSinkRecord record) throws SQLException { LOGGER.debug("Attempting to create table '{}'.", collectionId.toFullIdentiferString()); if (NONE.equals(config.getSchemaEvolutionMode())) { @@ -332,7 +302,7 @@ private TableDescriptor createTable(CollectionId collectionId, SinkRecordDescrip return readTable(collectionId); } - private TableDescriptor alterTableIfNeeded(CollectionId collectionId, SinkRecordDescriptor record) throws SQLException { + private TableDescriptor alterTableIfNeeded(CollectionId collectionId, JdbcSinkRecord record) throws SQLException { LOGGER.debug("Attempting to alter table '{}'.", collectionId.toFullIdentiferString()); if (!hasTable(collectionId)) { @@ -353,7 +323,7 @@ private TableDescriptor alterTableIfNeeded(CollectionId collectionId, SinkRecord LOGGER.debug("The follow fields are missing in the table: {}", missingFields); for (String missingFieldName : missingFields) { - final FieldDescriptor fieldDescriptor = record.getFields().get(missingFieldName); + final FieldDescriptor fieldDescriptor = record.allFields().get(missingFieldName); if (!fieldDescriptor.getSchema().isOptional() && fieldDescriptor.getSchema().defaultValue() == null) { throw new SQLException(String.format( "Cannot ALTER table '%s' because field '%s' is not optional but has no default value", @@ -381,14 +351,13 @@ private TableDescriptor alterTableIfNeeded(CollectionId collectionId, SinkRecord return readTable(collectionId); } - private String getSqlStatement(TableDescriptor table, SinkRecordDescriptor record) { - + private String getSqlStatement(TableDescriptor table, JdbcSinkRecord record) { if (!record.isDelete()) { switch (config.getInsertMode()) { case INSERT: return dialect.getInsertStatement(table, record); case UPSERT: - if (record.getKeyFieldNames().isEmpty()) { + if (record.keyFieldNames().isEmpty()) { throw new ConnectException("Cannot write to table " + table.getId().name() + " with no key fields defined."); } return dialect.getUpsertStatement(table, record); @@ -434,8 +403,8 @@ private boolean isRetriable(Throwable throwable) { return isRetriable(throwable.getCause()); } - public Optional getCollectionIdFromRecord(SinkRecord record) { - String tableName = this.config.getCollectionNamingStrategy().resolveCollectionName(config, record); + public Optional getCollectionIdFromRecord(DebeziumSinkRecord record) { + String tableName = this.config.getCollectionNamingStrategy().resolveCollectionName(record, config.getCollectionNameFormat()); if (tableName == null) { return Optional.empty(); } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcKafkaSinkRecord.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcKafkaSinkRecord.java new file mode 100644 index 00000000000..900816c7964 --- /dev/null +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcKafkaSinkRecord.java @@ -0,0 +1,233 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.jdbc; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Stream; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.sink.SinkRecord; + +import io.debezium.annotation.Immutable; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; +import io.debezium.connector.jdbc.dialect.DatabaseDialect; +import io.debezium.data.Envelope; +import io.debezium.sink.SinkConnectorConfig.PrimaryKeyMode; +import io.debezium.sink.filter.FieldFilterFactory.FieldNameFilter; + +/** + * An immutable representation of a {@link SinkRecord}. + * + * @author Chris Cranford + * @author rk3rn3r + */ +@Immutable +public class JdbcKafkaSinkRecord extends KafkaDebeziumSinkRecord implements JdbcSinkRecord { + + private final PrimaryKeyMode primaryKeyMode; + private final Set primaryKeyFields; + private final FieldNameFilter fieldsFilter; + private final DatabaseDialect dialect; + private final List keyFieldNames = new ArrayList<>(); + private final List nonKeyFieldNames = new ArrayList<>(); + private final Map allFields = new LinkedHashMap<>(); + + public JdbcKafkaSinkRecord(SinkRecord record, PrimaryKeyMode primaryKeyMode, Set primaryKeyFields, FieldNameFilter fieldsFilter, DatabaseDialect dialect) { + super(record); + + Objects.requireNonNull(primaryKeyMode, "The primary key mode must be provided."); + Objects.requireNonNull(record, "The sink record must be provided."); + + this.primaryKeyMode = primaryKeyMode; + this.primaryKeyFields = primaryKeyFields; + this.fieldsFilter = fieldsFilter; + this.dialect = dialect; + + final var flattened = isFlattened(); + final boolean truncated = !flattened && isTruncate(); + if (!truncated) { + readSinkRecordKeyData(flattened); + readSinkRecordNonKeyData(flattened); + } + } + + public List keyFieldNames() { + return keyFieldNames; + } + + public List getNonKeyFieldNames() { + return nonKeyFieldNames; + } + + public Map allFields() { + return allFields; + } + + private void readSinkRecordKeyData(boolean flattened) { + switch (primaryKeyMode) { + case NONE: + // does nothing + break; + case KAFKA: + applyKafkaCoordinatesAsPrimaryKey(); + break; + case RECORD_KEY: + applyRecordKeyAsPrimaryKey(); + break; + case RECORD_HEADER: + applyRecordHeaderAsPrimaryKey(); + break; + case RECORD_VALUE: + applyRecordValueAsPrimaryKey(flattened); + break; + default: + throw new ConnectException("Unexpected primary key mode: " + primaryKeyMode); + } + } + + private void readSinkRecordNonKeyData(boolean flattened) { + final Schema valueSchema = valueSchema(); + if (valueSchema != null) { + if (flattened) { + // In a flattened event type, it's safe to read the field names directly + // from the schema as this isn't a complex Debezium message type. + applyNonKeyFields(topicName(), valueSchema); + } + else { + // In a non-flattened event type, this is a complex Debezium type. + // We want to source the field names strictly from the 'after' block. + final Field after = valueSchema.field(Envelope.FieldName.AFTER); + if (after == null) { + throw new ConnectException("Received an unexpected message type that does not have an 'after' Debezium block"); + } + applyNonKeyFields(topicName(), after.schema()); + } + } + } + + private static final String CONNECT_TOPIC = "__connect_topic"; + private static final String CONNECT_PARTITION = "__connect_partition"; + private static final String CONNECT_OFFSET = "__connect_offset"; + + private void applyKafkaCoordinatesAsPrimaryKey() { + // CONNECT_TOPIC + keyFieldNames.add(CONNECT_TOPIC); + allFields.put(CONNECT_TOPIC, new FieldDescriptor(Schema.STRING_SCHEMA, CONNECT_TOPIC, true, dialect)); + + // CONNECT_PARTITION + keyFieldNames.add(CONNECT_PARTITION); + allFields.put(CONNECT_PARTITION, new FieldDescriptor(Schema.INT32_SCHEMA, CONNECT_PARTITION, true, dialect)); + + // CONNECT_OFFSET + keyFieldNames.add(CONNECT_OFFSET); + allFields.put(CONNECT_OFFSET, new FieldDescriptor(Schema.INT64_SCHEMA, CONNECT_OFFSET, true, dialect)); + } + + private void applyRecordKeyAsPrimaryKey() { + final Schema keySchema = keySchema(); + if (keySchema == null) { + throw new ConnectException("Configured primary key mode 'record_key' cannot have null schema"); + } + else if (keySchema.type().isPrimitive()) { + applyPrimitiveRecordKeyAsPrimaryKey(keySchema); + } + else if (Schema.Type.STRUCT.equals(keySchema.type())) { + applyRecordKeyAsPrimaryKey(topicName(), keySchema); + } + else { + throw new ConnectException("An unsupported record key schema type detected: " + keySchema.type()); + } + } + + private void applyRecordHeaderAsPrimaryKey() { + if (originalKafkaRecord.headers() == null || originalKafkaRecord.headers().isEmpty()) { + throw new ConnectException("Configured primary key mode 'record_header' cannot have null or empty schema"); + } + + final SchemaBuilder headerSchemaBuilder = SchemaBuilder.struct(); + originalKafkaRecord.headers().forEach((Header header) -> headerSchemaBuilder.field(header.key(), header.schema())); + final Schema headerSchema = headerSchemaBuilder.build(); + applyRecordKeyAsPrimaryKey(topicName(), headerSchema); + + } + + private void applyRecordValueAsPrimaryKey(boolean flattened) { + + final Schema valueSchema = valueSchema(); + if (valueSchema == null) { + throw new ConnectException("Configured primary key mode 'record_value' cannot have null schema"); + } + + Stream recordFields; + if (flattened) { + recordFields = valueSchema().fields().stream(); + } + else { + recordFields = ((Struct) value()).getStruct(Envelope.FieldName.AFTER).schema().fields().stream(); + } + + if (!primaryKeyFields.isEmpty()) { + recordFields = recordFields.filter(field -> primaryKeyFields.contains(field.name())); + } + + recordFields.forEach(field -> addKeyField(topicName(), field)); + } + + private void applyPrimitiveRecordKeyAsPrimaryKey(Schema keySchema) { + if (primaryKeyFields.isEmpty()) { + throw new ConnectException("The " + JdbcSinkConnectorConfig.PRIMARY_KEY_FIELDS + + " configuration must be specified when using a primitive key."); + } + addKeyField(primaryKeyFields.iterator().next(), keySchema); + } + + private void applyRecordKeyAsPrimaryKey(String topic, Schema keySchema) { + for (Field field : keySchema.fields()) { + if (primaryKeyFields.isEmpty() || primaryKeyFields.contains(field.name())) { + addKeyField(topic, field); + } + } + } + + private void addKeyField(String topic, Field field) { + if (fieldsFilter.matches(topic, field.name())) { + addKeyField(field.name(), field.schema()); + } + } + + private void addKeyField(String name, Schema schema) { + FieldDescriptor fieldDescriptor = new FieldDescriptor(schema, name, true, dialect); + keyFieldNames.add(fieldDescriptor.getName()); + allFields.put(fieldDescriptor.getName(), fieldDescriptor); + } + + private void applyNonKeyFields(String topic, Schema schema) { + for (Field field : schema.fields()) { + if (!keyFieldNames.contains(field.name())) { + if (fieldsFilter.matches(topic, field.name())) { + applyNonKeyField(field.name(), field.schema()); + } + } + } + } + + private void applyNonKeyField(String name, Schema schema) { + FieldDescriptor fieldDescriptor = new FieldDescriptor(schema, name, false, dialect); + nonKeyFieldNames.add(fieldDescriptor.getName()); + allFields.put(fieldDescriptor.getName(), fieldDescriptor); + } + +} diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorConfig.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorConfig.java index 12302d6a4a7..16a3abca951 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorConfig.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorConfig.java @@ -147,6 +147,7 @@ public class JdbcSinkConnectorConfig implements SinkConnectorConfig { "'insert' - the default mode, uses standard SQL insert statements; " + "'upsert' - uses upsert semantics for the database if its supported and requires setting primary.key.mode and primary.key.fields;" + "'update' - uses update semantics for the database if its supported."); + public static final Field DELETE_ENABLED_FIELD = SinkConnectorConfig.DELETE_ENABLED_FIELD .withValidation(JdbcSinkConnectorConfig::validateDeleteEnabled); diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorTask.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorTask.java index 2d96a733150..16a26da40d7 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorTask.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkConnectorTask.java @@ -28,7 +28,6 @@ import io.debezium.DebeziumException; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectResolver; -import io.debezium.sink.spi.ChangeEventSink; import io.debezium.util.Stopwatch; import io.debezium.util.Strings; @@ -54,7 +53,7 @@ private enum State { private final AtomicReference state = new AtomicReference<>(State.STOPPED); private final ReentrantLock stateLock = new ReentrantLock(); - private ChangeEventSink changeEventSink; + private JdbcChangeEventSink changeEventSink; private final Map offsets = new HashMap<>(); private Throwable previousPutException; diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkRecord.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkRecord.java new file mode 100644 index 00000000000..062dba81eb4 --- /dev/null +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/JdbcSinkRecord.java @@ -0,0 +1,127 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.jdbc; + +import java.util.List; +import java.util.Map; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.annotation.Immutable; +import io.debezium.connector.jdbc.dialect.DatabaseDialect; +import io.debezium.connector.jdbc.relational.ColumnDescriptor; +import io.debezium.connector.jdbc.type.Type; +import io.debezium.connector.jdbc.util.SchemaUtils; +import io.debezium.sink.DebeziumSinkRecord; + +/** + * @author rk3rn3r + */ +@Immutable +public interface JdbcSinkRecord extends DebeziumSinkRecord { + + List getNonKeyFieldNames(); + + Map allFields(); + + /** + * An immutable representation of a {@link Field} in a {@link SinkRecord}. + * + * @author Chris Cranford + */ + @Immutable + class FieldDescriptor { + + private static final Logger LOGGER = LoggerFactory.getLogger(FieldDescriptor.class); + + private final Schema schema; + private final String name; + private final String columnName; + private final boolean isKey; + private final Type type; + private final String typeName; + + // Lazily prepared + private String queryBinding; + + protected FieldDescriptor(Schema schema, String name, boolean isKey, DatabaseDialect dialect) { + this.schema = schema; + this.isKey = isKey; + + // These are cached here allowing them to be resolved once per record + this.type = dialect.getSchemaType(schema); + this.typeName = type.getTypeName(dialect, schema, isKey); + + this.name = name; + this.columnName = SchemaUtils.getSourceColumnName(schema).orElse(name); + + LOGGER.trace("Field [{}] with schema [{}]", this.name, schema.type()); + LOGGER.trace(" Type : {}", type.getClass().getName()); + LOGGER.trace(" Type Name : {}", typeName); + LOGGER.trace(" Optional : {}", schema.isOptional()); + + if (schema.parameters() != null && !schema.parameters().isEmpty()) { + LOGGER.trace(" Parameters: {}", schema.parameters()); + } + + if (schema.defaultValue() != null) { + LOGGER.trace(" Def. Value: {}", schema.defaultValue()); + } + } + + public Schema getSchema() { + return schema; + } + + public String getName() { + return name; + } + + public String getColumnName() { + return columnName; + } + + public boolean isKey() { + return isKey; + } + + public Type getType() { + return type; + } + + public String getTypeName() { + return typeName; + } + + public String getQueryBinding(ColumnDescriptor column, Object value) { + if (queryBinding == null) { + queryBinding = type.getQueryBinding(column, schema, value); + } + return queryBinding; + } + + public List bind(int startIndex, Object value) { + return type.bind(startIndex, schema, value); + } + + @Override + public String toString() { + return "FieldDescriptor{" + + "schema=" + schema + + ", name='" + name + '\'' + + ", key=" + isKey + + ", typeName='" + typeName + '\'' + + ", type=" + type + + ", columnName='" + columnName + '\'' + + '}'; + } + } + +} diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/PreparedStatementQueryBinder.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/PreparedStatementQueryBinder.java index a6c48d63b7c..f1f994b8a59 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/PreparedStatementQueryBinder.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/PreparedStatementQueryBinder.java @@ -21,7 +21,6 @@ public PreparedStatementQueryBinder(PreparedStatement binder) { @Override public void bind(ValueBindDescriptor valueBindDescriptor) { - try { if (valueBindDescriptor.getTargetSqlType() != null) { if (valueBindDescriptor.getTargetSqlType() == Types.ARRAY) { diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordBuffer.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordBuffer.java index f437477c60b..b68d8643a5f 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordBuffer.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordBuffer.java @@ -12,7 +12,7 @@ import org.apache.kafka.connect.data.Schema; /** - * A buffer of {@link SinkRecordDescriptor}. It contains the logic of when is the time to flush + * A buffer of {@link JdbcSinkRecord}. It contains the logic of when is the time to flush * * @author Mario Fiore Vitale */ @@ -21,31 +21,30 @@ public class RecordBuffer implements Buffer { private final JdbcSinkConnectorConfig connectorConfig; private Schema keySchema; private Schema valueSchema; - private final ArrayList records = new ArrayList<>(); + private final ArrayList records = new ArrayList<>(); public RecordBuffer(JdbcSinkConnectorConfig connectorConfig) { this.connectorConfig = connectorConfig; } - public List add(SinkRecordDescriptor recordDescriptor) { - - List flushed = new ArrayList<>(); + public List add(JdbcSinkRecord record) { + List flushed = new ArrayList<>(); boolean isSchemaChanged = false; if (records.isEmpty()) { - keySchema = recordDescriptor.getKeySchema(); - valueSchema = recordDescriptor.getValueSchema(); + keySchema = record.keySchema(); + valueSchema = record.valueSchema(); } - if (!Objects.equals(keySchema, recordDescriptor.getKeySchema()) || !Objects.equals(valueSchema, recordDescriptor.getValueSchema())) { - keySchema = recordDescriptor.getKeySchema(); - valueSchema = recordDescriptor.getValueSchema(); + if (!Objects.equals(keySchema, record.keySchema()) || !Objects.equals(valueSchema, record.valueSchema())) { + keySchema = record.keySchema(); + valueSchema = record.valueSchema(); flushed = flush(); isSchemaChanged = true; } - records.add(recordDescriptor); + records.add(record); if (isSchemaChanged) { // current record is already added in internal buffer after flush @@ -60,9 +59,9 @@ public List add(SinkRecordDescriptor recordDescriptor) { return flushed; } - public List flush() { + public List flush() { - List flushed = new ArrayList<>(records); + List flushed = new ArrayList<>(records); records.clear(); return flushed; diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordWriter.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordWriter.java index 7d6c8406388..47a95bffe07 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordWriter.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/RecordWriter.java @@ -41,8 +41,7 @@ public RecordWriter(SharedSessionContract session, QueryBinderResolver queryBind this.dialect = dialect; } - public void write(List records, String sqlStatement) { - + public void write(List records, String sqlStatement) { Stopwatch writeStopwatch = Stopwatch.reusable(); writeStopwatch.start(); final Transaction transaction = session.beginTransaction(); @@ -59,20 +58,18 @@ public void write(List records, String sqlStatement) { LOGGER.trace("[PERF] Total write execution time {}", writeStopwatch.durations()); } - private Work processBatch(List records, String sqlStatement) { - + private Work processBatch(List records, String sqlStatement) { return conn -> { - try (PreparedStatement prepareStatement = conn.prepareStatement(sqlStatement)) { QueryBinder queryBinder = queryBinderResolver.resolve(prepareStatement); Stopwatch allbindStopwatch = Stopwatch.reusable(); allbindStopwatch.start(); - for (SinkRecordDescriptor sinkRecordDescriptor : records) { + for (JdbcSinkRecord record : records) { Stopwatch singlebindStopwatch = Stopwatch.reusable(); singlebindStopwatch.start(); - bindValues(sinkRecordDescriptor, queryBinder); + bindValues(record, queryBinder); singlebindStopwatch.stop(); Stopwatch addBatchStopwatch = Stopwatch.reusable(); @@ -100,51 +97,48 @@ private Work processBatch(List records, String sqlStatemen }; } - private void bindValues(SinkRecordDescriptor sinkRecordDescriptor, QueryBinder queryBinder) { - + private void bindValues(JdbcSinkRecord record, QueryBinder queryBinder) { int index; - if (sinkRecordDescriptor.isDelete()) { - bindKeyValuesToQuery(sinkRecordDescriptor, queryBinder, 1); + if (record.isDelete()) { + bindKeyValuesToQuery(record, queryBinder, 1); return; } switch (config.getInsertMode()) { case INSERT: case UPSERT: - index = bindKeyValuesToQuery(sinkRecordDescriptor, queryBinder, 1); - bindNonKeyValuesToQuery(sinkRecordDescriptor, queryBinder, index); + index = bindKeyValuesToQuery(record, queryBinder, 1); + bindNonKeyValuesToQuery(record, queryBinder, index); break; case UPDATE: - index = bindNonKeyValuesToQuery(sinkRecordDescriptor, queryBinder, 1); - bindKeyValuesToQuery(sinkRecordDescriptor, queryBinder, index); + index = bindNonKeyValuesToQuery(record, queryBinder, 1); + bindKeyValuesToQuery(record, queryBinder, index); break; } } - private int bindKeyValuesToQuery(SinkRecordDescriptor record, QueryBinder query, int index) { - + private int bindKeyValuesToQuery(JdbcSinkRecord record, QueryBinder query, int index) { if (Objects.requireNonNull(config.getPrimaryKeyMode()) == JdbcSinkConnectorConfig.PrimaryKeyMode.KAFKA) { - query.bind(new ValueBindDescriptor(index++, record.getTopicName())); - query.bind(new ValueBindDescriptor(index++, record.getPartition())); - query.bind(new ValueBindDescriptor(index++, record.getOffset())); + query.bind(new ValueBindDescriptor(index++, record.topicName())); + query.bind(new ValueBindDescriptor(index++, record.partition())); + query.bind(new ValueBindDescriptor(index++, record.offset())); } else { final Struct keySource = record.getKeyStruct(config.getPrimaryKeyMode()); if (keySource != null) { - index = bindFieldValuesToQuery(record, query, index, keySource, record.getKeyFieldNames()); + index = bindFieldValuesToQuery(record, query, index, keySource, record.keyFieldNames()); } } return index; } - private int bindNonKeyValuesToQuery(SinkRecordDescriptor record, QueryBinder query, int index) { - return bindFieldValuesToQuery(record, query, index, record.getAfterStruct(), record.getNonKeyFieldNames()); + private int bindNonKeyValuesToQuery(JdbcSinkRecord record, QueryBinder query, int index) { + return bindFieldValuesToQuery(record, query, index, record.getPayload(), record.getNonKeyFieldNames()); } - private int bindFieldValuesToQuery(SinkRecordDescriptor record, QueryBinder query, int index, Struct source, List fields) { - + private int bindFieldValuesToQuery(JdbcSinkRecord record, QueryBinder query, int index, Struct source, List fields) { for (String fieldName : fields) { - final SinkRecordDescriptor.FieldDescriptor field = record.getFields().get(fieldName); + final JdbcSinkRecord.FieldDescriptor field = record.allFields().get(fieldName); Object value; if (field.getSchema().isOptional()) { diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/ReducedRecordBuffer.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/ReducedRecordBuffer.java index 6b1f42a9eeb..bda07eebaa8 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/ReducedRecordBuffer.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/ReducedRecordBuffer.java @@ -16,7 +16,7 @@ import org.apache.kafka.connect.errors.ConnectException; /** - * A reduced implementation buffer of {@link SinkRecordDescriptor}. + * A reduced implementation buffer of {@link JdbcSinkRecord}. * It reduces events in buffer before submit to external database. * * @author Gaurav Miglani @@ -27,32 +27,32 @@ public class ReducedRecordBuffer implements Buffer { private Schema keySchema; private Schema valueSchema; - private final Map records = new HashMap<>(); + private final Map records = new HashMap<>(); public ReducedRecordBuffer(JdbcSinkConnectorConfig connectorConfig) { this.connectorConfig = connectorConfig; } @Override - public List add(SinkRecordDescriptor recordDescriptor) { - List flushed = new ArrayList<>(); + public List add(JdbcSinkRecord record) { + List flushed = new ArrayList<>(); boolean isSchemaChanged = false; if (records.isEmpty()) { - keySchema = recordDescriptor.getKeySchema(); - valueSchema = recordDescriptor.getValueSchema(); + keySchema = record.keySchema(); + valueSchema = record.valueSchema(); } - if (!Objects.equals(keySchema, recordDescriptor.getKeySchema()) || !Objects.equals(valueSchema, recordDescriptor.getValueSchema())) { - keySchema = recordDescriptor.getKeySchema(); - valueSchema = recordDescriptor.getValueSchema(); + if (!Objects.equals(keySchema, record.keySchema()) || !Objects.equals(valueSchema, record.valueSchema())) { + keySchema = record.keySchema(); + valueSchema = record.valueSchema(); flushed = flush(); isSchemaChanged = true; } - Struct keyStruct = recordDescriptor.getKeyStruct(connectorConfig.getPrimaryKeyMode()); + Struct keyStruct = record.getKeyStruct(connectorConfig.getPrimaryKeyMode()); if (keyStruct != null) { - records.put(keyStruct, recordDescriptor); + records.put(keyStruct, record); } else { throw new ConnectException("No struct-based primary key defined for record key/value, reduction buffer require struct based primary key"); @@ -72,8 +72,8 @@ public List add(SinkRecordDescriptor recordDescriptor) { } @Override - public List flush() { - List flushed = new ArrayList<>(records.values()); + public List flush() { + List flushed = new ArrayList<>(records.values()); records.clear(); return flushed; } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/SinkRecordDescriptor.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/SinkRecordDescriptor.java deleted file mode 100644 index 0ba7fe38699..00000000000 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/SinkRecordDescriptor.java +++ /dev/null @@ -1,485 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.jdbc; - -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Stream; - -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.header.Header; -import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.debezium.annotation.Immutable; -import io.debezium.connector.jdbc.dialect.DatabaseDialect; -import io.debezium.connector.jdbc.relational.ColumnDescriptor; -import io.debezium.connector.jdbc.type.Type; -import io.debezium.connector.jdbc.util.SchemaUtils; -import io.debezium.data.Envelope; -import io.debezium.data.Envelope.Operation; -import io.debezium.sink.SinkConnectorConfig.PrimaryKeyMode; -import io.debezium.sink.filter.FieldFilterFactory; -import io.debezium.sink.filter.FieldFilterFactory.FieldNameFilter; - -/** - * An immutable representation of a {@link SinkRecord}. - * - * @author Chris Cranford - */ -@Immutable -public class SinkRecordDescriptor { - private final SinkRecord record; - private final String topicName; - private final List keyFieldNames; - private final List nonKeyFieldNames; - private final Map fields; - private final boolean flattened; - - private SinkRecordDescriptor(SinkRecord record, String topicName, List keyFieldNames, List nonKeyFieldNames, - Map fields, boolean flattened) { - this.record = record; - this.topicName = topicName; - this.keyFieldNames = keyFieldNames; - this.nonKeyFieldNames = nonKeyFieldNames; - this.fields = fields; - this.flattened = flattened; - } - - public String getTopicName() { - return topicName; - } - - public Integer getPartition() { - return record.kafkaPartition(); - } - - public long getOffset() { - return record.kafkaOffset(); - } - - public List getKeyFieldNames() { - return keyFieldNames; - } - - public List getNonKeyFieldNames() { - return nonKeyFieldNames; - } - - public Map getFields() { - return fields; - } - - public boolean isDebeziumSinkRecord() { - return !flattened; - } - - public boolean isTombstone() { - // NOTE - // Debezium TOMBSTONE has both value and valueSchema to null, instead the ExtractNewRecordState SMT with delete.handling.mode=none will generate - // a record only with value null that by JDBC connector is treated as a flattened delete. See isDelete method. - return record.value() == null && record.valueSchema() == null; - } - - public boolean isDelete() { - if (!isDebeziumSinkRecord()) { - return record.value() == null; - } - else if (record.value() != null) { - final Struct value = (Struct) record.value(); - return Operation.DELETE.equals(Operation.forCode(value.getString(Envelope.FieldName.OPERATION))); - } - return false; - } - - public boolean isTruncate() { - if (isDebeziumSinkRecord()) { - final Struct value = (Struct) record.value(); - return Operation.TRUNCATE.equals(Operation.forCode(value.getString(Envelope.FieldName.OPERATION))); - } - return false; - } - - public Schema getKeySchema() { - return record.keySchema(); - } - - public Schema getValueSchema() { - return record.valueSchema(); - } - - public Struct getKeyStruct(PrimaryKeyMode primaryKeyMode) { - if (!getKeyFieldNames().isEmpty()) { - switch (primaryKeyMode) { - case RECORD_KEY: - final Schema keySchema = record.keySchema(); - if (keySchema != null && Schema.Type.STRUCT.equals(keySchema.type())) { - return (Struct) record.key(); - } - else { - throw new ConnectException("No struct-based primary key defined for record key."); - } - case RECORD_VALUE: - final Schema valueSchema = record.valueSchema(); - if (valueSchema != null && Schema.Type.STRUCT.equals(valueSchema.type())) { - return getAfterStruct(); - } - else { - throw new ConnectException("No struct-based primary key defined for record value."); - } - - case RECORD_HEADER: - final SchemaBuilder headerSchemaBuilder = SchemaBuilder.struct(); - record.headers().forEach((Header header) -> headerSchemaBuilder.field(header.key(), header.schema())); - - final Schema headerSchema = headerSchemaBuilder.build(); - final Struct headerStruct = new Struct(headerSchema); - record.headers().forEach((Header header) -> headerStruct.put(header.key(), header.value())); - return headerStruct; - } - } - return null; - } - - public Struct getAfterStruct() { - if (isDebeziumSinkRecord()) { - return ((Struct) record.value()).getStruct(Envelope.FieldName.AFTER); - } - else { - return ((Struct) record.value()); - } - } - - public static Builder builder() { - return new Builder(); - } - - /** - * An immutable representation of a {@link Field} in a {@link SinkRecord}. - * - * @author Chris Cranford - */ - @Immutable - public static class FieldDescriptor { - - private static final Logger LOGGER = LoggerFactory.getLogger(FieldDescriptor.class); - - private final Schema schema; - private final String name; - private final String columnName; - private final boolean key; - private final Type type; - private final DatabaseDialect dialect; - private final String typeName; - - // Lazily prepared - private String queryBinding; - - private FieldDescriptor(Schema schema, String name, boolean key, DatabaseDialect dialect) { - this.schema = schema; - this.key = key; - this.dialect = dialect; - - // These are cached here allowing them to be resolved once per record - this.type = dialect.getSchemaType(schema); - this.typeName = type.getTypeName(dialect, schema, key); - - this.name = name; - this.columnName = SchemaUtils.getSourceColumnName(schema).orElse(name); - - LOGGER.trace("Field [{}] with schema [{}]", this.name, schema.type()); - LOGGER.trace(" Type : {}", type.getClass().getName()); - LOGGER.trace(" Type Name : {}", typeName); - LOGGER.trace(" Optional : {}", schema.isOptional()); - - if (schema.parameters() != null && !schema.parameters().isEmpty()) { - LOGGER.trace(" Parameters: {}", schema.parameters()); - } - - if (schema.defaultValue() != null) { - LOGGER.trace(" Def. Value: {}", schema.defaultValue()); - } - } - - public Schema getSchema() { - return schema; - } - - public String getName() { - return name; - } - - public String getColumnName() { - return columnName; - } - - public boolean isKey() { - return key; - } - - public Type getType() { - return type; - } - - public String getTypeName() { - return typeName; - } - - public String getQueryBinding(ColumnDescriptor column, Object value) { - if (queryBinding == null) { - queryBinding = type.getQueryBinding(column, schema, value); - } - return queryBinding; - } - - public List bind(int startIndex, Object value) { - return type.bind(startIndex, schema, value); - } - - @Override - public String toString() { - return "FieldDescriptor{" + - "schema=" + schema + - ", name='" + name + '\'' + - ", key=" + key + - ", typeName='" + typeName + '\'' + - ", type=" + type + - ", columnName='" + columnName + '\'' + - '}'; - } - } - - public static class Builder { - - private static final String CONNECT_TOPIC = "__connect_topic"; - private static final String CONNECT_PARTITION = "__connect_partition"; - private static final String CONNECT_OFFSET = "__connect_offset"; - - // External contributed builder state - private PrimaryKeyMode primaryKeyMode; - private Set primaryKeyFields; - private FieldNameFilter fieldFilter = FieldFilterFactory.DEFAULT_FILTER; - private SinkRecord sinkRecord; - private DatabaseDialect dialect; - - // Internal build state - private final List keyFieldNames = new ArrayList<>(); - private final List nonKeyFieldNames = new ArrayList<>(); - private final Map allFields = new LinkedHashMap<>(); - - public Builder withDialect(DatabaseDialect dialect) { - this.dialect = dialect; - return this; - } - - public Builder withPrimaryKeyFields(Set primaryKeyFields) { - this.primaryKeyFields = primaryKeyFields; - return this; - } - - public Builder withPrimaryKeyMode(PrimaryKeyMode primaryKeyMode) { - this.primaryKeyMode = primaryKeyMode; - return this; - } - - public Builder withSinkRecord(SinkRecord record) { - this.sinkRecord = record; - return this; - } - - public Builder withFieldFilter(FieldNameFilter fieldFilter) { - this.fieldFilter = fieldFilter; - return this; - } - - public SinkRecordDescriptor build() { - Objects.requireNonNull(primaryKeyMode, "The primary key mode must be provided."); - Objects.requireNonNull(sinkRecord, "The sink record must be provided."); - - final boolean flattened = notTombstone(sinkRecord) && isFlattened(sinkRecord); - final boolean truncated = !flattened && isTruncateEvent(sinkRecord); - if (!truncated) { - readSinkRecordKeyData(sinkRecord, flattened); - readSinkRecordNonKeyData(sinkRecord, flattened); - } - - return new SinkRecordDescriptor(sinkRecord, sinkRecord.topic(), keyFieldNames, nonKeyFieldNames, allFields, flattened); - } - - private boolean isFlattened(SinkRecord record) { - return record.valueSchema().name() == null || !record.valueSchema().name().contains("Envelope"); - } - - private boolean notTombstone(SinkRecord record) { - - return record.value() != null || record.valueSchema() != null; - } - - private boolean isTruncateEvent(SinkRecord record) { - return notTombstone(record) - && Operation.TRUNCATE.equals(Operation.forCode(((Struct) record.value()).getString(Envelope.FieldName.OPERATION))); - } - - private void readSinkRecordKeyData(SinkRecord record, boolean flattened) { - switch (primaryKeyMode) { - case NONE: - // does nothing - break; - case KAFKA: - applyKafkaCoordinatesAsPrimaryKey(); - break; - case RECORD_KEY: - applyRecordKeyAsPrimaryKey(record); - break; - case RECORD_HEADER: - applyRecordHeaderAsPrimaryKey(record); - break; - case RECORD_VALUE: - applyRecordValueAsPrimaryKey(record, flattened); - break; - default: - throw new ConnectException("Unexpected primary key mode: " + primaryKeyMode); - } - } - - private void applyKafkaCoordinatesAsPrimaryKey() { - // CONNECT_TOPIC - keyFieldNames.add(CONNECT_TOPIC); - allFields.put(CONNECT_TOPIC, new FieldDescriptor(Schema.STRING_SCHEMA, CONNECT_TOPIC, true, dialect)); - - // CONNECT_PARTITION - keyFieldNames.add(CONNECT_PARTITION); - allFields.put(CONNECT_PARTITION, new FieldDescriptor(Schema.INT32_SCHEMA, CONNECT_PARTITION, true, dialect)); - - // CONNECT_OFFSET - keyFieldNames.add(CONNECT_OFFSET); - allFields.put(CONNECT_OFFSET, new FieldDescriptor(Schema.INT64_SCHEMA, CONNECT_OFFSET, true, dialect)); - } - - private void applyRecordKeyAsPrimaryKey(SinkRecord record) { - final Schema keySchema = record.keySchema(); - if (keySchema == null) { - throw new ConnectException("Configured primary key mode 'record_key' cannot have null schema"); - } - else if (keySchema.type().isPrimitive()) { - applyPrimitiveRecordKeyAsPrimaryKey(keySchema); - } - else if (Schema.Type.STRUCT.equals(keySchema.type())) { - applyRecordKeyAsPrimaryKey(record.topic(), keySchema); - } - else { - throw new ConnectException("An unsupported record key schema type detected: " + keySchema.type()); - } - } - - private void applyRecordHeaderAsPrimaryKey(SinkRecord record) { - if (record.headers() == null || record.headers().isEmpty()) { - throw new ConnectException("Configured primary key mode 'record_header' cannot have null or empty schema"); - } - - final SchemaBuilder headerSchemaBuilder = SchemaBuilder.struct(); - record.headers().forEach((Header header) -> headerSchemaBuilder.field(header.key(), header.schema())); - final Schema headerSchema = headerSchemaBuilder.build(); - applyRecordKeyAsPrimaryKey(record.topic(), headerSchema); - - } - - private void applyRecordValueAsPrimaryKey(SinkRecord record, boolean flattened) { - - final Schema valueSchema = record.valueSchema(); - if (valueSchema == null) { - throw new ConnectException("Configured primary key mode 'record_value' cannot have null schema"); - } - - Stream recordFields; - if (flattened) { - recordFields = record.valueSchema().fields().stream(); - } - else { - recordFields = ((Struct) record.value()).getStruct(Envelope.FieldName.AFTER).schema().fields().stream(); - } - - if (!primaryKeyFields.isEmpty()) { - recordFields = recordFields.filter(field -> primaryKeyFields.contains(field.name())); - } - - recordFields.forEach(field -> addKeyField(record.topic(), field)); - } - - private void applyPrimitiveRecordKeyAsPrimaryKey(Schema keySchema) { - if (primaryKeyFields.isEmpty()) { - throw new ConnectException("The " + JdbcSinkConnectorConfig.PRIMARY_KEY_FIELDS + - " configuration must be specified when using a primitive key."); - } - addKeyField(primaryKeyFields.iterator().next(), keySchema); - } - - private void applyRecordKeyAsPrimaryKey(String topic, Schema keySchema) { - for (Field field : keySchema.fields()) { - if (primaryKeyFields.isEmpty() || primaryKeyFields.contains(field.name())) { - addKeyField(topic, field); - } - } - } - - private void addKeyField(String topic, Field field) { - if (fieldFilter.matches(topic, field.name())) { - addKeyField(field.name(), field.schema()); - } - } - - private void addKeyField(String name, Schema schema) { - FieldDescriptor fieldDescriptor = new FieldDescriptor(schema, name, true, dialect); - keyFieldNames.add(fieldDescriptor.getName()); - allFields.put(fieldDescriptor.getName(), fieldDescriptor); - } - - private void readSinkRecordNonKeyData(SinkRecord record, boolean flattened) { - final Schema valueSchema = record.valueSchema(); - if (valueSchema != null) { - if (flattened) { - // In a flattened event type, it's safe to read the field names directly - // from the schema as this isn't a complex Debezium message type. - applyNonKeyFields(record.topic(), valueSchema); - } - else { - // In a non-flattened event type, this is a complex Debezium type. - // We want to source the field names strictly from the 'after' block. - final Field after = valueSchema.field(Envelope.FieldName.AFTER); - if (after == null) { - throw new ConnectException("Received an unexpected message type that does not have an 'after' Debezium block"); - } - applyNonKeyFields(record.topic(), after.schema()); - } - } - } - - private void applyNonKeyFields(String topic, Schema schema) { - for (Field field : schema.fields()) { - if (!keyFieldNames.contains(field.name())) { - if (fieldFilter.matches(topic, field.name())) { - applyNonKeyField(field.name(), field.schema()); - } - } - } - } - - private void applyNonKeyField(String name, Schema schema) { - FieldDescriptor fieldDescriptor = new FieldDescriptor(schema, name, false, dialect); - nonKeyFieldNames.add(fieldDescriptor.getName()); - allFields.put(fieldDescriptor.getName(), fieldDescriptor); - } - } -} diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/DatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/DatabaseDialect.java index bcd15884c97..edee7fa9970 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/DatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/DatabaseDialect.java @@ -15,8 +15,8 @@ import org.hibernate.dialect.DatabaseVersion; import org.hibernate.engine.jdbc.Size; -import io.debezium.connector.jdbc.SinkRecordDescriptor; -import io.debezium.connector.jdbc.SinkRecordDescriptor.FieldDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; +import io.debezium.connector.jdbc.JdbcSinkRecord.FieldDescriptor; import io.debezium.connector.jdbc.ValueBindDescriptor; import io.debezium.connector.jdbc.relational.ColumnDescriptor; import io.debezium.connector.jdbc.relational.TableDescriptor; @@ -74,7 +74,7 @@ public interface DatabaseDialect { * @param table the relational table model, should not be {@code null} * @return a collection of field names that are missing from the database table, can be {@code empty}. */ - Set resolveMissingFields(SinkRecordDescriptor record, TableDescriptor table); + Set resolveMissingFields(JdbcSinkRecord record, TableDescriptor table); /** * Construct a {@code CREATE TABLE} statement specific for this dialect based on the provided record. @@ -83,7 +83,7 @@ public interface DatabaseDialect { * @param collectionId the tableidentifier to be used, should not be {@code null} * @return the create table SQL statement to be executed, never {@code null} */ - String getCreateTableStatement(SinkRecordDescriptor record, CollectionId collectionId); + String getCreateTableStatement(JdbcSinkRecord record, CollectionId collectionId); /** * Gets the prefix used before adding column-clauses in {@code ALTER TABLE} statements. @@ -129,7 +129,7 @@ public interface DatabaseDialect { * @return the alter table SQL statement to be executed, never {@code null} * @throws IllegalArgumentException if called with an empty set of missing fields */ - String getAlterTableStatement(TableDescriptor table, SinkRecordDescriptor record, Set missingFields); + String getAlterTableStatement(TableDescriptor table, JdbcSinkRecord record, Set missingFields); /** * Construct a {@code INSERT INTO} statement specific for this dialect. @@ -138,7 +138,7 @@ public interface DatabaseDialect { * @param record the current sink record being processed, should not be {@code null} * @return the insert SQL statement to be executed, never {@code null} */ - String getInsertStatement(TableDescriptor table, SinkRecordDescriptor record); + String getInsertStatement(TableDescriptor table, JdbcSinkRecord record); /** * Construct a {@code UPSERT} statement specific for this dialect. @@ -147,7 +147,7 @@ public interface DatabaseDialect { * @param record the current sink record being processed, should not be {@code null} * @return the upsert SQL statement to be executed, never {@code null} */ - String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record); + String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record); /** * Construct a {@code UPDATE} statement specific for this dialect. @@ -156,7 +156,7 @@ public interface DatabaseDialect { * @param record the current sink record being processed, should not be {@code null} * @return the update SQL statement to be executed, never {@code null} */ - String getUpdateStatement(TableDescriptor table, SinkRecordDescriptor record); + String getUpdateStatement(TableDescriptor table, JdbcSinkRecord record); /** * Construct a {@code DELETE} statement specific for this dialect. @@ -165,7 +165,7 @@ public interface DatabaseDialect { * @param record the current sink record being processed, should not be {@code null} * @return the delete SQL statement to be executed, never {@code null} */ - String getDeleteStatement(TableDescriptor table, SinkRecordDescriptor record); + String getDeleteStatement(TableDescriptor table, JdbcSinkRecord record); /** * Construct a {@code TRUNCATE} statement specific for this dialect. diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/GeneralDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/GeneralDatabaseDialect.java index f6d53b75395..e21de99e7f6 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/GeneralDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/GeneralDatabaseDialect.java @@ -46,8 +46,8 @@ import io.debezium.DebeziumException; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; -import io.debezium.connector.jdbc.SinkRecordDescriptor.FieldDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; +import io.debezium.connector.jdbc.JdbcSinkRecord.FieldDescriptor; import io.debezium.connector.jdbc.ValueBindDescriptor; import io.debezium.connector.jdbc.naming.ColumnNamingStrategy; import io.debezium.connector.jdbc.relational.ColumnDescriptor; @@ -222,10 +222,10 @@ public TableDescriptor readTable(Connection connection, CollectionId collectionI } @Override - public Set resolveMissingFields(SinkRecordDescriptor record, TableDescriptor table) { + public Set resolveMissingFields(JdbcSinkRecord record, TableDescriptor table) { final Set missingFields = new HashSet<>(); - for (FieldDescriptor field : record.getFields().values()) { + for (FieldDescriptor field : record.allFields().values()) { String columnName = resolveColumnName(field); if (!table.hasColumn(columnName)) { missingFields.add(field.getName()); @@ -251,15 +251,15 @@ protected String resolveColumnName(FieldDescriptor field) { } @Override - public String getCreateTableStatement(SinkRecordDescriptor record, CollectionId collectionId) { + public String getCreateTableStatement(JdbcSinkRecord record, CollectionId collectionId) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("CREATE TABLE "); builder.append(getQualifiedTableName(collectionId)); builder.append(" ("); // First handle key columns - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> { - final FieldDescriptor field = record.getFields().get(name); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> { + final FieldDescriptor field = record.allFields().get(name); final String columnName = toIdentifier(resolveColumnName(field)); final String columnType = field.getTypeName(); @@ -278,10 +278,10 @@ public String getCreateTableStatement(SinkRecordDescriptor record, CollectionId return columnSpec.toString(); }); - if (!record.getKeyFieldNames().isEmpty()) { + if (!record.keyFieldNames().isEmpty()) { builder.append(", PRIMARY KEY("); - builder.appendList(", ", record.getKeyFieldNames(), (name) -> { - final FieldDescriptor field = record.getFields().get(name); + builder.appendList(", ", record.keyFieldNames(), (name) -> { + final FieldDescriptor field = record.allFields().get(name); return toIdentifier(columnNamingStrategy.resolveColumnName(field.getColumnName())); }); builder.append(")"); @@ -318,14 +318,14 @@ public String getAlterTableColumnDelimiter() { } @Override - public String getAlterTableStatement(TableDescriptor table, SinkRecordDescriptor record, Set missingFields) { + public String getAlterTableStatement(TableDescriptor table, JdbcSinkRecord record, Set missingFields) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("ALTER TABLE "); builder.append(getQualifiedTableName(table.getId())); builder.append(" "); builder.append(getAlterTablePrefix()); builder.appendList(getAlterTableColumnDelimiter(), missingFields, (name) -> { - final FieldDescriptor field = record.getFields().get(name); + final FieldDescriptor field = record.allFields().get(name); final StringBuilder addColumnSpec = new StringBuilder(); addColumnSpec.append(getAlterTableColumnPrefix()); addColumnSpec.append(" "); @@ -342,18 +342,18 @@ public String getAlterTableStatement(TableDescriptor table, SinkRecordDescriptor } @Override - public String getInsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getInsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("INSERT INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); builder.append(")"); @@ -361,35 +361,35 @@ public String getInsertStatement(TableDescriptor table, SinkRecordDescriptor rec } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { throw new UnsupportedOperationException("Upsert configurations are not supported for this dialect"); } @Override - public String getUpdateStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpdateStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("UPDATE "); builder.append(getQualifiedTableName(table.getId())); builder.append(" SET "); builder.appendList(", ", record.getNonKeyFieldNames(), (name) -> columnNameEqualsBinding(name, table, record)); - if (!record.getKeyFieldNames().isEmpty()) { + if (!record.keyFieldNames().isEmpty()) { builder.append(" WHERE "); - builder.appendList(" AND ", record.getKeyFieldNames(), (name) -> columnNameEqualsBinding(name, table, record)); + builder.appendList(" AND ", record.keyFieldNames(), (name) -> columnNameEqualsBinding(name, table, record)); } return builder.build(); } @Override - public String getDeleteStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getDeleteStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("DELETE FROM "); builder.append(getQualifiedTableName(table.getId())); - if (!record.getKeyFieldNames().isEmpty()) { + if (!record.keyFieldNames().isEmpty()) { builder.append(" WHERE "); - builder.appendList(" AND ", record.getKeyFieldNames(), (name) -> columnNameEqualsBinding(name, table, record)); + builder.appendList(" AND ", record.keyFieldNames(), (name) -> columnNameEqualsBinding(name, table, record)); } return builder.build(); @@ -687,8 +687,8 @@ protected void addColumnDefaultValue(FieldDescriptor field, StringBuilder column } } - protected String columnQueryBindingFromField(String fieldName, TableDescriptor table, SinkRecordDescriptor record) { - final FieldDescriptor field = record.getFields().get(fieldName); + protected String columnQueryBindingFromField(String fieldName, TableDescriptor table, JdbcSinkRecord record) { + final FieldDescriptor field = record.allFields().get(fieldName); final String columnName = resolveColumnName(field); final ColumnDescriptor column = table.getColumnByName(columnName); @@ -699,10 +699,10 @@ protected String columnQueryBindingFromField(String fieldName, TableDescriptor t else { value = getColumnValueFromKeyField(fieldName, record, columnName); } - return record.getFields().get(fieldName).getQueryBinding(column, value); + return record.allFields().get(fieldName).getQueryBinding(column, value); } - private Object getColumnValueFromKeyField(String fieldName, SinkRecordDescriptor record, String columnName) { + private Object getColumnValueFromKeyField(String fieldName, JdbcSinkRecord record, String columnName) { Object value; if (connectorConfig.getPrimaryKeyMode() == JdbcSinkConnectorConfig.PrimaryKeyMode.KAFKA) { value = getColumnValueForKafkaKeyMode(columnName, record); @@ -714,29 +714,25 @@ private Object getColumnValueFromKeyField(String fieldName, SinkRecordDescriptor return value; } - private Object getColumnValueFromValueField(String fieldName, SinkRecordDescriptor record) { - return record.getAfterStruct().get(fieldName); + private Object getColumnValueFromValueField(String fieldName, JdbcSinkRecord record) { + return record.getPayload().get(fieldName); } - private Object getColumnValueForKafkaKeyMode(String columnName, SinkRecordDescriptor record) { - switch (columnName) { - case "__connect_topic": - return record.getTopicName(); - case "__connect_partition": - return record.getPartition(); - case "__connect_offset": - return record.getOffset(); - default: - return null; - } + private Object getColumnValueForKafkaKeyMode(String columnName, JdbcSinkRecord record) { + return switch (columnName) { + case "__connect_topic" -> record.topicName(); + case "__connect_partition" -> record.partition(); + case "__connect_offset" -> record.offset(); + default -> null; + }; } - protected String columnNameFromField(String fieldName, SinkRecordDescriptor record) { - final FieldDescriptor field = record.getFields().get(fieldName); + protected String columnNameFromField(String fieldName, JdbcSinkRecord record) { + final FieldDescriptor field = record.allFields().get(fieldName); return toIdentifier(resolveColumnName(field)); } - protected String columnNameFromField(String fieldName, String prefix, SinkRecordDescriptor record) { + protected String columnNameFromField(String fieldName, String prefix, JdbcSinkRecord record) { return prefix + columnNameFromField(fieldName, record); } @@ -780,11 +776,11 @@ protected String getQualifiedTableName(CollectionId collectionId) { return toIdentifier(collectionId.name()); } - private String columnNameEqualsBinding(String fieldName, TableDescriptor table, SinkRecordDescriptor record) { - final FieldDescriptor field = record.getFields().get(fieldName); + private String columnNameEqualsBinding(String fieldName, TableDescriptor table, JdbcSinkRecord record) { + final FieldDescriptor field = record.allFields().get(fieldName); final String columnName = resolveColumnName(field); final ColumnDescriptor column = table.getColumnByName(columnName); - return toIdentifier(columnName) + "=" + field.getQueryBinding(column, record.getAfterStruct()); + return toIdentifier(columnName) + "=" + field.getQueryBinding(column, record.getPayload()); } private static boolean isColumnNullable(String columnName, Collection primaryKeyColumnNames, int nullability) { diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/db2/Db2DatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/db2/Db2DatabaseDialect.java index 22f35f5ef2d..95d9415d9d9 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/db2/Db2DatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/db2/Db2DatabaseDialect.java @@ -16,8 +16,8 @@ import org.hibernate.dialect.Dialect; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; -import io.debezium.connector.jdbc.SinkRecordDescriptor.FieldDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; +import io.debezium.connector.jdbc.JdbcSinkRecord.FieldDescriptor; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.GeneralDatabaseDialect; @@ -139,31 +139,31 @@ public String getTimestampNegativeInfinityValue() { } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("merge into "); builder.append(getQualifiedTableName(table.getId())); builder.append(" using (values("); - builder.appendLists(record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); + builder.appendLists(record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); builder.append(")) as DAT("); - builder.appendLists(record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") on "); - builder.appendList(" AND ", record.getKeyFieldNames(), (name) -> getMergeDatClause(name, table, record)); + builder.appendList(" AND ", record.keyFieldNames(), (name) -> getMergeDatClause(name, table, record)); if (!record.getNonKeyFieldNames().isEmpty()) { builder.append(" WHEN MATCHED THEN UPDATE SET "); builder.appendList(", ", record.getNonKeyFieldNames(), (name) -> getMergeDatClause(name, table, record)); } builder.append(" WHEN NOT MATCHED THEN INSERT("); - builder.appendLists(",", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(",", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") values ("); - builder.appendLists(",", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> "DAT." + columnNameFromField(name, record)); + builder.appendLists(",", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> "DAT." + columnNameFromField(name, record)); builder.append(")"); return builder.build(); } - private String getMergeDatClause(String fieldName, TableDescriptor table, SinkRecordDescriptor record) { + private String getMergeDatClause(String fieldName, TableDescriptor table, JdbcSinkRecord record) { final String columnName = columnNameFromField(fieldName, record); return toIdentifier(table.getId()) + "." + columnName + "=DAT." + columnName; } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MariaDbDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MariaDbDatabaseDialect.java index 48bef79c4e5..8719286cb3c 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MariaDbDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MariaDbDatabaseDialect.java @@ -14,7 +14,7 @@ import org.slf4j.LoggerFactory; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.SqlStatementBuilder; @@ -55,18 +55,18 @@ private MariaDbDatabaseDialect(JdbcSinkConnectorConfig config, SessionFactory se * but not followed by MariaDB yet. */ @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("INSERT INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), name -> columnNameFromField(name, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), name -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), name -> columnQueryBindingFromField(name, table, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), name -> columnQueryBindingFromField(name, table, record)); builder.append(") "); final List updateColumnNames = record.getNonKeyFieldNames().isEmpty() - ? record.getKeyFieldNames() + ? record.keyFieldNames() : record.getNonKeyFieldNames(); builder.append("ON DUPLICATE KEY UPDATE "); diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MySqlDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MySqlDatabaseDialect.java index 6ef6aa0aa99..c7ab7a4b350 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MySqlDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/mysql/MySqlDatabaseDialect.java @@ -24,7 +24,7 @@ import org.hibernate.exception.LockAcquisitionException; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.GeneralDatabaseDialect; @@ -161,18 +161,18 @@ public String getAlterTablePrefix() { } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("INSERT INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); builder.append(") "); final List updateColumnNames = record.getNonKeyFieldNames().isEmpty() - ? record.getKeyFieldNames() + ? record.keyFieldNames() : record.getNonKeyFieldNames(); if (getDatabaseVersion().isSameOrAfter(8, 0, 20)) { @@ -200,7 +200,7 @@ public Set> getCommunicationExceptions() { } @Override - protected void addColumnDefaultValue(SinkRecordDescriptor.FieldDescriptor field, StringBuilder columnSpec) { + protected void addColumnDefaultValue(JdbcSinkRecord.FieldDescriptor field, StringBuilder columnSpec) { final String fieldType = field.getTypeName(); if (!Strings.isNullOrBlank(fieldType)) { if (NO_DEFAULT_VALUE_TYPES.contains(fieldType.toLowerCase())) { diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/oracle/OracleDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/oracle/OracleDatabaseDialect.java index 3a5bdbccda5..aa0d5b35b43 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/oracle/OracleDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/oracle/OracleDatabaseDialect.java @@ -18,7 +18,7 @@ import org.hibernate.dialect.OracleDialect; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.GeneralDatabaseDialect; @@ -94,24 +94,24 @@ public boolean isNegativeScaleAllowed() { } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("MERGE INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" USING (SELECT "); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record) + " " + columnNameFromField(name, record)); builder.append(" FROM dual) ").append("INCOMING ON ("); - builder.appendList(" AND ", record.getKeyFieldNames(), (name) -> getUpsertIncomingClause(name, table, record)); + builder.appendList(" AND ", record.keyFieldNames(), (name) -> getUpsertIncomingClause(name, table, record)); builder.append(")"); if (!record.getNonKeyFieldNames().isEmpty()) { builder.append(" WHEN MATCHED THEN UPDATE SET "); builder.appendList(",", record.getNonKeyFieldNames(), (name) -> getUpsertIncomingClause(name, table, record)); } builder.append(" WHEN NOT MATCHED THEN INSERT ("); - builder.appendLists(",", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(",", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(",", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> columnNameFromField(name, "INCOMING.", record)); + builder.appendLists(",", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> columnNameFromField(name, "INCOMING.", record)); builder.append(")"); return builder.build(); } @@ -176,7 +176,7 @@ protected String resolveColumnNameFromField(String fieldName) { return columnName; } - private String getUpsertIncomingClause(String fieldName, TableDescriptor table, SinkRecordDescriptor record) { + private String getUpsertIncomingClause(String fieldName, TableDescriptor table, JdbcSinkRecord record) { final String columnName = columnNameFromField(fieldName, record); return toIdentifier(table.getId()) + "." + columnName + "=INCOMING." + columnName; } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/GeometryType.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/GeometryType.java index 75b6a9cc025..ab0e6b5d667 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/GeometryType.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/GeometryType.java @@ -13,6 +13,7 @@ import io.debezium.connector.jdbc.type.AbstractGeoType; import io.debezium.connector.jdbc.type.Type; import io.debezium.data.geometry.Geometry; +import io.debezium.sink.SinkConnectorConfig; public class GeometryType extends AbstractGeoType { @@ -23,10 +24,14 @@ public class GeometryType extends AbstractGeoType { String postgisSchema; @Override - public void configure(JdbcSinkConnectorConfig config, DatabaseDialect dialect) { + public void configure(SinkConnectorConfig config, DatabaseDialect dialect) { super.configure(config, dialect); - - this.postgisSchema = config.getPostgresPostgisSchema(); + if (config instanceof JdbcSinkConnectorConfig jdbcConfig) { + this.postgisSchema = jdbcConfig.getPostgresPostgisSchema(); + } + else { + this.postgisSchema = "public"; + } } @Override diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/PostgresDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/PostgresDatabaseDialect.java index 1cd0b3e5e01..5853546d017 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/PostgresDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/postgres/PostgresDatabaseDialect.java @@ -20,7 +20,7 @@ import org.hibernate.dialect.PostgreSQLDialect; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.GeneralDatabaseDialect; @@ -97,16 +97,16 @@ public String getAlterTableColumnPrefix() { } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("INSERT INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" ("); - builder.appendLists(",", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(",", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(",", record.getKeyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); + builder.appendLists(",", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnQueryBindingFromField(name, table, record)); builder.append(") ON CONFLICT ("); - builder.appendList(",", record.getKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendList(",", record.keyFieldNames(), (name) -> columnNameFromField(name, record)); if (record.getNonKeyFieldNames().isEmpty()) { builder.append(") DO NOTHING"); } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/BitType.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/BitType.java index 6ffdf8c918d..cd39ab0b9f4 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/BitType.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/BitType.java @@ -63,7 +63,6 @@ public String getTypeName(DatabaseDialect dialect, Schema schema, boolean key) { @Override public List bind(int index, Schema schema, Object value) { - if (value == null) { return List.of(new ValueBindDescriptor(index, null)); } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/SqlServerDatabaseDialect.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/SqlServerDatabaseDialect.java index 6aba18ba5fa..79fc5c87458 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/SqlServerDatabaseDialect.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/dialect/sqlserver/SqlServerDatabaseDialect.java @@ -12,7 +12,7 @@ import org.hibernate.dialect.SQLServerDialect; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; -import io.debezium.connector.jdbc.SinkRecordDescriptor; +import io.debezium.connector.jdbc.JdbcSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.dialect.DatabaseDialectProvider; import io.debezium.connector.jdbc.dialect.GeneralDatabaseDialect; @@ -49,7 +49,7 @@ private SqlServerDatabaseDialect(JdbcSinkConnectorConfig config, SessionFactory } @Override - public String getInsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getInsertStatement(TableDescriptor table, JdbcSinkRecord record) { String insertStatement = super.getInsertStatement(table, record); return wrapWithIdentityInsert(table, insertStatement); } @@ -125,15 +125,15 @@ public String getTimestampNegativeInfinityValue() { } @Override - public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor record) { + public String getUpsertStatement(TableDescriptor table, JdbcSinkRecord record) { final SqlStatementBuilder builder = new SqlStatementBuilder(); builder.append("MERGE INTO "); builder.append(getQualifiedTableName(table.getId())); builder.append(" WITH (HOLDLOCK) AS TARGET USING (SELECT "); - builder.appendLists(", ", record.getKeyFieldNames(), record.getNonKeyFieldNames(), + builder.appendLists(", ", record.keyFieldNames(), record.getNonKeyFieldNames(), (name) -> columnNameFromField(name, columnQueryBindingFromField(name, table, record) + " AS ", record)); builder.append(") AS INCOMING ON ("); - builder.appendList(" AND ", record.getKeyFieldNames(), (name) -> { + builder.appendList(" AND ", record.keyFieldNames(), (name) -> { final String columnName = columnNameFromField(name, record); return "TARGET." + columnName + "=INCOMING." + columnName; }); @@ -148,9 +148,9 @@ public String getUpsertStatement(TableDescriptor table, SinkRecordDescriptor rec } builder.append(" WHEN NOT MATCHED THEN INSERT ("); - builder.appendLists(", ", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> columnNameFromField(name, record)); + builder.appendLists(", ", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> columnNameFromField(name, record)); builder.append(") VALUES ("); - builder.appendLists(",", record.getNonKeyFieldNames(), record.getKeyFieldNames(), (name) -> columnNameFromField(name, "INCOMING.", record)); + builder.appendLists(",", record.getNonKeyFieldNames(), record.keyFieldNames(), (name) -> columnNameFromField(name, "INCOMING.", record)); builder.append(")"); builder.append(";"); // SQL server requires this to be terminated this way. diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractTemporalType.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractTemporalType.java index 5b6ef8152ee..c684e81994d 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractTemporalType.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractTemporalType.java @@ -11,8 +11,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.dialect.DatabaseDialect; +import io.debezium.sink.SinkConnectorConfig; /** * An abstract base class for all temporal implementations of {@link Type}. @@ -26,7 +26,7 @@ public abstract class AbstractTemporalType extends AbstractType { private TimeZone databaseTimeZone; @Override - public void configure(JdbcSinkConnectorConfig config, DatabaseDialect dialect) { + public void configure(SinkConnectorConfig config, DatabaseDialect dialect) { super.configure(config, dialect); final String databaseTimeZone = config.useTimeZone(); diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractType.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractType.java index 0867cfda3bf..740582d530e 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractType.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/AbstractType.java @@ -12,11 +12,11 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; -import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.ValueBindDescriptor; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.relational.ColumnDescriptor; import io.debezium.connector.jdbc.util.SchemaUtils; +import io.debezium.sink.SinkConnectorConfig; /** * An abstract implementation of {@link Type}, which all types should extend. @@ -28,7 +28,7 @@ public abstract class AbstractType implements Type { private DatabaseDialect dialect; @Override - public void configure(JdbcSinkConnectorConfig config, DatabaseDialect dialect) { + public void configure(SinkConnectorConfig config, DatabaseDialect dialect) { this.dialect = dialect; } diff --git a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/Type.java b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/Type.java index 615d9ce1268..2c88c1bfb13 100644 --- a/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/Type.java +++ b/debezium-connector-jdbc/src/main/java/io/debezium/connector/jdbc/type/Type.java @@ -9,10 +9,10 @@ import org.apache.kafka.connect.data.Schema; -import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.ValueBindDescriptor; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.relational.ColumnDescriptor; +import io.debezium.sink.SinkConnectorConfig; /** * A type represents a relational column type used for query abd parameter binding. @@ -29,7 +29,7 @@ public interface Type { * @param config the JDBC sink connector's configuration, should not be {@code null} * @param dialect the database dialect, should not be {@code null} */ - void configure(JdbcSinkConnectorConfig config, DatabaseDialect dialect); + void configure(SinkConnectorConfig config, DatabaseDialect dialect); /** * Returns the names that this type will be mapped as. diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/AbstractRecordBufferTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/AbstractRecordBufferTest.java index 1fe2ccdce67..fbb99d74b22 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/AbstractRecordBufferTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/AbstractRecordBufferTest.java @@ -6,9 +6,9 @@ package io.debezium.connector.jdbc; -import org.apache.kafka.connect.sink.SinkRecord; import org.jetbrains.annotations.NotNull; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.util.SinkRecordFactory; @@ -16,21 +16,20 @@ class AbstractRecordBufferTest { protected DatabaseDialect dialect; - protected @NotNull SinkRecordDescriptor createRecord(SinkRecord kafkaRecord, JdbcSinkConnectorConfig config) { - return SinkRecordDescriptor.builder() - .withSinkRecord(kafkaRecord) - .withDialect(dialect) - .withPrimaryKeyMode(config.getPrimaryKeyMode()) - .withPrimaryKeyFields(config.getPrimaryKeyFields()) - .withFieldFilter(config.getFieldFilter()) - .build(); + protected @NotNull JdbcKafkaSinkRecord createRecord(KafkaDebeziumSinkRecord record, JdbcSinkConnectorConfig config) { + return new JdbcKafkaSinkRecord( + record.getOriginalKafkaRecord(), + config.getPrimaryKeyMode(), + config.getPrimaryKeyFields(), + config.getFieldFilter(), + dialect); } - protected @NotNull SinkRecordDescriptor createRecordNoPkFields(SinkRecordFactory factory, byte i, JdbcSinkConnectorConfig config) { + protected @NotNull JdbcKafkaSinkRecord createRecordNoPkFields(SinkRecordFactory factory, byte i, JdbcSinkConnectorConfig config) { return createRecord(factory.createRecord("topic", i), config); } - protected @NotNull SinkRecordDescriptor createRecordPkFieldId(SinkRecordFactory factory, byte i, JdbcSinkConnectorConfig config) { + protected @NotNull JdbcKafkaSinkRecord createRecordPkFieldId(SinkRecordFactory factory, byte i, JdbcSinkConnectorConfig config) { return createRecord(factory.createRecord("topic", i), config); } diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/CollectionNamingStrategyTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/CollectionNamingStrategyTest.java index 405c041c0f2..66db970562b 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/CollectionNamingStrategyTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/CollectionNamingStrategyTest.java @@ -10,7 +10,6 @@ import java.util.Map; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -28,12 +27,13 @@ */ @Tag("UnitTests") public class CollectionNamingStrategyTest { + @Test public void testDefaultTableNamingStrategy() { final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(Map.of()); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - assertThat(strategy.resolveCollectionName(config, factory.createRecord("database.schema.table"))) + assertThat(strategy.resolveCollectionName(factory.createRecord("database.schema.table"), config.getCollectionNameFormat())) .isEqualTo("database_schema_table"); } @@ -42,7 +42,7 @@ public void testTableNamingStrategyWithTableNameFormat() { final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "kafka_${topic}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - assertThat(strategy.resolveCollectionName(config, factory.createRecord("database.schema.table"))) + assertThat(strategy.resolveCollectionName(factory.createRecord("database.schema.table"), config.getCollectionNameFormat())) .isEqualTo("kafka_database_schema_table"); } @@ -52,7 +52,7 @@ public void testTableNamingStrategyWithPrependedSchema() { final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "SYS.${topic}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - assertThat(strategy.resolveCollectionName(config, factory.createRecord("database.schema.table"))) + assertThat(strategy.resolveCollectionName(factory.createRecord("database.schema.table"), config.getCollectionNameFormat())) .isEqualTo("SYS.database_schema_table"); } @@ -62,8 +62,8 @@ public void testDefaultTableNamingStrategyWithDebeziumSource() { Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "source_${source.db}_${source.schema}_${source.table}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - SinkRecord record = factory.createRecord("database.schema.table", (byte) 1, "database1", "schema1", "table1"); - assertThat(strategy.resolveCollectionName(config, record)) + assertThat(strategy.resolveCollectionName(factory.createRecord("database.schema.table", (byte) 1, "database1", "schema1", "table1"), + config.getCollectionNameFormat())) .isEqualTo("source_database1_schema1_table1"); } @@ -72,8 +72,8 @@ public void testDefaultTableNamingStrategyWithInvalidSourceField() { final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "source_${source.invalid}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - SinkRecord record = factory.createRecord("database.schema.table", (byte) 1, "database1", "schema1", "table1"); - Assertions.assertThrows(DataException.class, () -> strategy.resolveCollectionName(config, record)); + Assertions.assertThrows(DataException.class, () -> strategy + .resolveCollectionName(factory.createRecord("database.schema.table", (byte) 1, "database1", "schema1", "table1"), config.getCollectionNameFormat())); } @Test @@ -82,8 +82,7 @@ public void testDefaultTableNamingStrategyWithDebeziumSourceAndTombstone() { Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "source_${source.db}_${source.schema}_${source.table}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - SinkRecord record = factory.tombstoneRecord("database.schema.table"); - assertThat(strategy.resolveCollectionName(config, record)).isNull(); + assertThat(strategy.resolveCollectionName(factory.tombstoneRecord("database.schema.table"), config.getCollectionNameFormat())).isNull(); } @Test @@ -91,7 +90,7 @@ public void testDefaultTableNamingStrategyWithTopicAndTombstone() { final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(Map.of(JdbcSinkConnectorConfig.COLLECTION_NAME_FORMAT, "kafka_${topic}")); final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); final DefaultCollectionNamingStrategy strategy = new DefaultCollectionNamingStrategy(); - SinkRecord record = factory.tombstoneRecord("database.schema.table"); - assertThat(strategy.resolveCollectionName(config, record)).isEqualTo("kafka_database_schema_table"); + assertThat(strategy.resolveCollectionName(factory.tombstoneRecord("database.schema.table"), config.getCollectionNameFormat())) + .isEqualTo("kafka_database_schema_table"); } } diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/RecordBufferTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/RecordBufferTest.java index 9d1a0f3cea5..c45b514e8b1 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/RecordBufferTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/RecordBufferTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.sink.SinkRecord; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; @@ -31,6 +30,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.junit.jupiter.SinkRecordFactoryArgumentsProvider; import io.debezium.connector.jdbc.type.Type; @@ -60,13 +60,13 @@ void correctlyBuffer(SinkRecordFactory factory) { RecordBuffer recordBuffer = new RecordBuffer(config); - List sinkRecords = IntStream.range(0, 10) + List sinkRecords = IntStream.range(0, 10) .mapToObj(i -> createRecordNoPkFields(factory, (byte) i, config)) .collect(Collectors.toList()); - List> batches = sinkRecords.stream().map(recordBuffer::add) + List> batches = sinkRecords.stream().map(recordBuffer::add) .filter(not(List::isEmpty)) - .collect(Collectors.toList()); + .toList(); assertThat(batches.size()).isEqualTo(2); @@ -81,11 +81,11 @@ void keySchemaChange(SinkRecordFactory factory) { RecordBuffer recordBuffer = new RecordBuffer(config); - List sinkRecords = IntStream.range(0, 3) + List sinkRecords = IntStream.range(0, 3) .mapToObj(i -> createRecordNoPkFields(factory, (byte) i, config)) .collect(Collectors.toList()); - SinkRecord sinkRecordWithDifferentKeySchema = factory.updateBuilder() + KafkaDebeziumSinkRecord sinkRecordWithDifferentKeySchema = factory.updateBuilder() .name("prefix") .topic("topic") .keySchema(factory.keySchema(UnaryOperator.identity(), Schema.INT16_SCHEMA)) @@ -99,9 +99,9 @@ void keySchemaChange(SinkRecordFactory factory) { sinkRecords.add(createRecord(sinkRecordWithDifferentKeySchema, config)); - List> batches = sinkRecords.stream().map(recordBuffer::add) + List> batches = sinkRecords.stream().map(recordBuffer::add) .filter(not(List::isEmpty)) - .collect(Collectors.toList()); + .toList(); assertThat(batches.size()).isEqualTo(1); @@ -116,11 +116,11 @@ void valueSchemaChange(SinkRecordFactory factory) { RecordBuffer recordBuffer = new RecordBuffer(config); - List sinkRecords = IntStream.range(0, 3) + List sinkRecords = IntStream.range(0, 3) .mapToObj(i -> createRecordPkFieldId(factory, (byte) i, config)) .collect(Collectors.toList()); - SinkRecord sinkRecordWithDifferentValueSchema = factory.updateBuilder() + KafkaDebeziumSinkRecord sinkRecordWithDifferentValueSchema = factory.updateBuilder() .name("prefix") .topic("topic") .keySchema(factory.basicKeySchema()) @@ -134,9 +134,9 @@ void valueSchemaChange(SinkRecordFactory factory) { sinkRecords.add(createRecord(sinkRecordWithDifferentValueSchema, config)); - List> batches = sinkRecords.stream().map(recordBuffer::add) + List> batches = sinkRecords.stream().map(recordBuffer::add) .filter(not(List::isEmpty)) - .collect(Collectors.toList()); + .toList(); assertThat(batches.size()).isEqualTo(1); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/ReducedRecordBufferTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/ReducedRecordBufferTest.java index ce061eadd37..45d2d5db0e9 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/ReducedRecordBufferTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/ReducedRecordBufferTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; @@ -33,6 +32,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.dialect.DatabaseDialect; import io.debezium.connector.jdbc.junit.jupiter.SinkRecordFactoryArgumentsProvider; import io.debezium.connector.jdbc.type.Type; @@ -84,11 +84,11 @@ void correctlyBuffer(SinkRecordFactory factory) { ReducedRecordBuffer reducedRecordBuffer = new ReducedRecordBuffer(config); - List sinkRecords = IntStream.range(0, 10) + List sinkRecords = IntStream.range(0, 10) .mapToObj(i -> createRecordPkFieldId(factory, (byte) i, config)) .collect(Collectors.toList()); - List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) + List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) .filter(not(List::isEmpty)) .toList(); @@ -105,11 +105,11 @@ void keySchemaChange(SinkRecordFactory factory) { ReducedRecordBuffer reducedRecordBuffer = new ReducedRecordBuffer(config); - List sinkRecords = IntStream.range(0, 3) + List sinkRecords = IntStream.range(0, 3) .mapToObj(i -> createRecordPkFieldId(factory, (byte) i, config)) .collect(Collectors.toList()); - SinkRecord sinkRecordWithDifferentKeySchema = factory.updateBuilder() + KafkaDebeziumSinkRecord sinkRecordWithDifferentKeySchema = factory.updateBuilder() .name("prefix") .topic("topic") .keySchema(factory.keySchema(UnaryOperator.identity(), Schema.INT16_SCHEMA)) @@ -123,7 +123,7 @@ void keySchemaChange(SinkRecordFactory factory) { sinkRecords.add(createRecord(sinkRecordWithDifferentKeySchema, config)); - List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) + List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) .filter(not(List::isEmpty)) .toList(); @@ -140,11 +140,11 @@ void valueSchemaChange(SinkRecordFactory factory) { ReducedRecordBuffer reducedRecordBuffer = new ReducedRecordBuffer(config); - List sinkRecords = IntStream.range(0, 3) + List sinkRecords = IntStream.range(0, 3) .mapToObj(i -> createRecordPkFieldId(factory, (byte) i, config)) .collect(Collectors.toList()); - SinkRecord sinkRecordWithDifferentValueSchema = factory.updateBuilder() + KafkaDebeziumSinkRecord sinkRecordWithDifferentValueSchema = factory.updateBuilder() .name("prefix") .topic("topic") .keySchema(factory.basicKeySchema()) @@ -158,7 +158,7 @@ void valueSchemaChange(SinkRecordFactory factory) { sinkRecords.add(createRecord(sinkRecordWithDifferentValueSchema, config)); - List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) + List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) .filter(not(List::isEmpty)) .toList(); @@ -175,11 +175,11 @@ void correctlyBufferWithDuplicate(SinkRecordFactory factory) { ReducedRecordBuffer reducedRecordBuffer = new ReducedRecordBuffer(config); - List sinkRecords = IntStream.range(0, 10) + List sinkRecords = IntStream.range(0, 10) .mapToObj(i -> createRecordPkFieldId(factory, (byte) (i % 2 == 0 ? i : i - 1), config)) .collect(Collectors.toList()); - List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) + List> batches = sinkRecords.stream().map(reducedRecordBuffer::add) .filter(not(List::isEmpty)) .toList(); @@ -195,11 +195,11 @@ void raiseExceptionWithoutPrimaryKey(SinkRecordFactory factory) { ReducedRecordBuffer reducedRecordBuffer = new ReducedRecordBuffer(config); - List sinkRecords = IntStream.range(0, 10) + List sinkRecords = IntStream.range(0, 10) .mapToObj(i -> createRecordNoPkFields(factory, (byte) i, config)) .collect(Collectors.toList()); - Stream> batchesFilter = sinkRecords.stream().map(reducedRecordBuffer::add) + Stream> batchesFilter = sinkRecords.stream().map(reducedRecordBuffer::add) .filter(not(List::isEmpty)); Exception thrown = Assertions.assertThrows(ConnectException.class, batchesFilter::toList); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/e2e/AbstractJdbcSinkPipelineIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/e2e/AbstractJdbcSinkPipelineIT.java index b58ff4697a1..0e31ec6bdba 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/e2e/AbstractJdbcSinkPipelineIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/e2e/AbstractJdbcSinkPipelineIT.java @@ -42,6 +42,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.config.CommonConnectorConfig.BinaryHandlingMode; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.InsertMode; @@ -2851,7 +2852,8 @@ protected void registerSourceConnector(Source source, String tableName) { } private String getSinkTable(SinkRecord record, Sink sink) { - final String sinkTableName = collectionNamingStrategy.resolveCollectionName(getCurrentSinkConfig(), record); + final String sinkTableName = collectionNamingStrategy.resolveCollectionName(new KafkaDebeziumSinkRecord(record), + getCurrentSinkConfig().getCollectionNameFormat()); // When quoted identifiers is not enabled, PostgreSQL saves table names as lower-case return sink.getType().is(SinkType.POSTGRES) ? sinkTableName.toLowerCase() : sinkTableName; } diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkDeleteEnabledTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkDeleteEnabledTest.java index 5cbb9e26ea9..4345c409634 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkDeleteEnabledTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkDeleteEnabledTest.java @@ -9,12 +9,12 @@ import java.util.List; import java.util.Map; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; import io.debezium.connector.jdbc.junit.TestHelper; @@ -48,7 +48,7 @@ public void testShouldNotDeleteRowWhenDeletesDisabled(SinkRecordFactory factory) final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); consume(factory.deleteRecord(topicName)); @@ -73,7 +73,7 @@ public void testShouldDeleteRowWhenDeletesEnabled(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); consume(factory.deleteRecord(topicName)); @@ -99,7 +99,7 @@ public void testShouldDeleteRowWhenDeletesEnabledUsingSubsetOfRecordKeyFields(Si final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); consume(createRecord); consume(factory.deleteRecordMultipleKeyColumns(topicName)); @@ -124,7 +124,7 @@ public void testShouldHandleRowDeletionWhenRowDoesNotExist(SinkRecordFactory fac final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord deleteRecord = factory.deleteRecord(topicName); + final KafkaDebeziumSinkRecord deleteRecord = factory.deleteRecord(topicName); consume(deleteRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(deleteRecord)); @@ -149,7 +149,7 @@ public void testShouldHandleRowDeletionWhenRowDoesNotExistUsingSubsetOfRecordKey final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord deleteRecord = factory.deleteRecordMultipleKeyColumns(topicName); + final KafkaDebeziumSinkRecord deleteRecord = factory.deleteRecordMultipleKeyColumns(topicName); consume(deleteRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(deleteRecord)); @@ -175,7 +175,7 @@ public void testShouldSkipTombstoneRecord(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord deleteRecord = factory.deleteRecord(topicName); + final KafkaDebeziumSinkRecord deleteRecord = factory.deleteRecord(topicName); // Switching the normal order for test purpose. // If the delete record is not processed mean that the tombstone generated an error consume(factory.tombstoneRecord(topicName)); @@ -202,7 +202,7 @@ public void testShouldSkipTruncateRecord(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); consume(factory.truncateRecord(topicName)); @@ -228,7 +228,7 @@ public void testShouldHandleTruncateRecord(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); consume(factory.truncateRecord(topicName)); @@ -259,9 +259,9 @@ public void testShouldHandleCreateRecordsAfterTruncateRecord(SinkRecordFactory f final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - SinkRecord firstRecord = factory.createRecord(topicName, (byte) 1); - SinkRecord truncateRecord = factory.truncateRecord(topicName); - SinkRecord secondRecord = factory.createRecord(topicName, (byte) 2); + KafkaDebeziumSinkRecord firstRecord = factory.createRecord(topicName, (byte) 1); + KafkaDebeziumSinkRecord truncateRecord = factory.truncateRecord(topicName); + KafkaDebeziumSinkRecord secondRecord = factory.createRecord(topicName, (byte) 2); consume(firstRecord); consume(truncateRecord); @@ -292,8 +292,8 @@ public void testShouldFlushUpdateBufferWhenDelete(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord deleteRecord = factory.deleteRecord(topicName); - List records = new ArrayList(); + final KafkaDebeziumSinkRecord deleteRecord = factory.deleteRecord(topicName); + List records = new ArrayList<>(); records.add(factory.createRecord(topicName, (byte) 2)); records.add(factory.createRecord(topicName, (byte) 1)); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkInsertModeTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkInsertModeTest.java index 6aef6a2b58a..a283d8f0ea8 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkInsertModeTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkInsertModeTest.java @@ -13,12 +13,12 @@ import java.util.Map; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.InsertMode; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; @@ -58,7 +58,7 @@ public void testInsertModeInsertWithNoPrimaryKey(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); consume(factory.createRecordNoKey(topicName)); @@ -83,7 +83,7 @@ public void testInsertModeInsertWithPrimaryKeyModeKafka(SinkRecordFactory factor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); consume(factory.createRecord(topicName)); @@ -111,7 +111,7 @@ public void testInsertModeInsertWithPrimaryKeyModeRecordKey(SinkRecordFactory fa final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 2)); @@ -137,7 +137,7 @@ public void testInsertModeInsertWithPrimaryKeyModeRecordValue(SinkRecordFactory final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 2)); @@ -190,7 +190,7 @@ public void testInsertModeUpsertWithPrimaryKeyModeKafka(SinkRecordFactory factor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 1)); @@ -218,7 +218,7 @@ public void testInsertModeUpsertWithPrimaryKeyModeRecordKey(SinkRecordFactory fa final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 1)); @@ -244,7 +244,7 @@ public void testInsertModeUpsertWithPrimaryKeyModeRecordValue(SinkRecordFactory final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 1)); @@ -269,7 +269,7 @@ public void testInsertModeUpdateWithNoPrimaryKey(SinkRecordFactory factory) { final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); // No changes detected because there is no existing record. @@ -294,7 +294,7 @@ public void testInsertModeUpdateWithPrimaryKeyModeKafka(SinkRecordFactory factor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); // No changes detected because there is no existing record. @@ -322,7 +322,7 @@ public void testInsertModeUpdateWithPrimaryKeyModeRecordKey(SinkRecordFactory fa final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); // No changes detected because there is no existing record. @@ -348,7 +348,7 @@ public void testInsertModeUpdateWithPrimaryKeyModeRecordValue(SinkRecordFactory final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); // No changes detected because there is no existing record. @@ -374,7 +374,7 @@ public void testRecordDefaultValueUsedOnlyWithRequiredFieldWithNullValue(SinkRec final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue(topicName, + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, List.of("optional_with_default_null_value"), List.of(SchemaBuilder.string().defaultValue("default").optional().build()), diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkPrimaryKeyModeTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkPrimaryKeyModeTest.java index ce2f1142a05..3865158331d 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkPrimaryKeyModeTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkPrimaryKeyModeTest.java @@ -18,6 +18,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; import io.debezium.connector.jdbc.junit.TestHelper; @@ -49,7 +50,7 @@ public void testRecordWithNoPrimaryKeyColumnsWithPrimaryKeyModeNone(SinkRecordFa final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -76,7 +77,7 @@ public void testRecordWithNoPrimaryKeyColumnsWithPrimaryKeyModeKafka(SinkRecordF final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -106,7 +107,7 @@ public void testRecordWithPrimaryKeyColumnWithPrimaryKeyModeKafka(SinkRecordFact final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -136,7 +137,7 @@ public void testRecordWithPrimaryKeyColumnWithPrimaryKeyModeRecordKey(SinkRecord final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -170,7 +171,7 @@ public void testRecordWithPrimaryKeyColumnsWithPrimaryKeyModeRecordKey(SinkRecor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -197,10 +198,8 @@ public void testRecordWithPrimaryKeyColumnWithPrimaryKeyModeRecordHeader(SinkRec final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - SinkRecord createRecord = factory.createRecord(topicName); - createRecord = new SinkRecord(createRecord.topic(), createRecord.kafkaPartition(), null, null, createRecord.valueSchema(), createRecord.value(), - createRecord.kafkaOffset()); - createRecord.headers().addInt("id", 1); + KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); + createRecord.getOriginalKafkaRecord().headers().addInt("id", 1); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -234,14 +233,15 @@ public void testRecordWithPrimaryKeyColumnsWithPrimaryKeyModeRecordHeader(SinkRe final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - SinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); - createRecord = new SinkRecord(createRecord.topic(), createRecord.kafkaPartition(), null, null, createRecord.valueSchema(), createRecord.value(), - createRecord.kafkaOffset()); - createRecord.headers().addInt("id1", 1); - createRecord.headers().addInt("id2", 10); - consume(createRecord); + KafkaDebeziumSinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); + SinkRecord kafkaSinkRecord = new SinkRecord(createRecord.topicName(), createRecord.partition(), null, null, createRecord.valueSchema(), createRecord.value(), + createRecord.offset()); + kafkaSinkRecord.headers().addInt("id1", 1); + kafkaSinkRecord.headers().addInt("id2", 10); + KafkaDebeziumSinkRecord kafkaSinkRecordWithHeader = new KafkaDebeziumSinkRecord(kafkaSinkRecord); + consume(kafkaSinkRecordWithHeader); - final String destinationTableName = destinationTableName(createRecord); + final String destinationTableName = destinationTableName(kafkaSinkRecordWithHeader); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName); tableAssert.exists().hasNumberOfColumns(3); @@ -266,7 +266,7 @@ public void testRecordWithNoPrimaryKeyColumnsWithPrimaryKeyModeRecordValue(SinkR final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -292,7 +292,7 @@ public void testRecordWithPrimaryKeyColumnWithPrimaryKeyModeRecordValueWithNoFie final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -320,7 +320,7 @@ public void testRecordWithPrimaryKeyColumnWithPrimaryKeyModeRecordValue(SinkReco final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -348,7 +348,7 @@ public void testRecordWithPrimaryKeyColumnsWithPrimaryKeyModeRecordValue(SinkRec final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); @@ -376,7 +376,7 @@ public void testRecordWithPrimaryKeyColumnsWithPrimaryKeyModeRecordValueWithSubs final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordMultipleKeyColumns(topicName); consume(createRecord); final String destinationTableName = destinationTableName(createRecord); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSaveConvertedCloudEventTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSaveConvertedCloudEventTest.java index f50fbe1384a..95c019cf691 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSaveConvertedCloudEventTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSaveConvertedCloudEventTest.java @@ -12,13 +12,13 @@ import java.util.Map; import java.util.stream.Collectors; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.fest.assertions.Index; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; import io.debezium.connector.jdbc.junit.TestHelper; @@ -59,8 +59,8 @@ public void testSaveConvertedCloudEventRecordFromJson(SinkRecordFactory factory) final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("json"), null); - final SinkRecord convertedRecord = transform.apply(cloudEventRecord); + final KafkaDebeziumSinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("json"), null); + final KafkaDebeziumSinkRecord convertedRecord = new KafkaDebeziumSinkRecord(transform.apply(cloudEventRecord.getOriginalKafkaRecord())); consume(convertedRecord); final String destinationTableName = destinationTableName(convertedRecord); @@ -96,8 +96,8 @@ public void testSaveConvertedCloudEventRecordFromAvro(SinkRecordFactory factory) final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("avro"), null); - final SinkRecord convertedRecord = transform.apply(cloudEventRecord); + final KafkaDebeziumSinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("avro"), null); + final KafkaDebeziumSinkRecord convertedRecord = new KafkaDebeziumSinkRecord(transform.apply(cloudEventRecord.getOriginalKafkaRecord())); consume(convertedRecord); final String destinationTableName = destinationTableName(convertedRecord); @@ -134,8 +134,8 @@ public void testSaveConvertedCloudEventRecordFromAvroWithCloudEventsSchemaCustom final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("avro"), "TestCESchemaCustomName"); - final SinkRecord convertedRecord = transform.apply(cloudEventRecord); + final KafkaDebeziumSinkRecord cloudEventRecord = factory.cloudEventRecord(topicName, SerializerType.withName("avro"), "TestCESchemaCustomName"); + final KafkaDebeziumSinkRecord convertedRecord = new KafkaDebeziumSinkRecord(transform.apply(cloudEventRecord.getOriginalKafkaRecord())); consume(convertedRecord); final String destinationTableName = destinationTableName(convertedRecord); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSchemaEvolutionTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSchemaEvolutionTest.java index 47a368cb6b4..6f289a2cd03 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSchemaEvolutionTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkSchemaEvolutionTest.java @@ -13,12 +13,12 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; import io.debezium.connector.jdbc.junit.TestHelper; @@ -115,7 +115,7 @@ public void testTableCreatedOnCreateRecordWithDefaultInsertMode(SinkRecordFactor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(createRecord)); @@ -137,7 +137,7 @@ public void testTableCreatedOnUpdateRecordWithDefaultInsertMode(SinkRecordFactor final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord updateRecord = factory.updateRecord(topicName); + final KafkaDebeziumSinkRecord updateRecord = factory.updateRecord(topicName); consume(updateRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(updateRecord)); @@ -160,7 +160,7 @@ public void testTableCreatedOnDeleteRecordWithDefaultInsertModeAndDeleteEnabled( final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord deleteRecord = factory.deleteRecord(topicName); + final KafkaDebeziumSinkRecord deleteRecord = factory.deleteRecord(topicName); consume(deleteRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(deleteRecord)); @@ -182,10 +182,10 @@ public void testTableCreatedThenAlteredWithNewColumn(SinkRecordFactory factory) final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); - final SinkRecord updateRecord = factory.updateBuilder() + final KafkaDebeziumSinkRecord updateRecord = factory.updateBuilder() .name("prefix") .topic(topicName) .keySchema(factory.basicKeySchema()) @@ -227,10 +227,10 @@ public void testTableCreatedThenNotAlteredWithRemovedColumn(SinkRecordFactory fa final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecord(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName); consume(createRecord); - final SinkRecord updateRecord = factory.updateBuilder() + final KafkaDebeziumSinkRecord updateRecord = factory.updateBuilder() .name("prefix") .topic(topicName) .keySchema(factory.basicKeySchema()) @@ -264,7 +264,7 @@ public void testNonKeyColumnTypeResolutionFromKafkaSchemaType(SinkRecordFactory final String topicName = topicName("server1", "schema", tableName); // Create record, optionals provided. - final SinkRecord createRecord = factory.createBuilder() + final KafkaDebeziumSinkRecord createRecord = factory.createBuilder() .name("prefix") .topic(topicName) .keySchema(factory.basicKeySchema()) @@ -337,7 +337,7 @@ public void testNonKeyColumnTypeResolutionFromKafkaSchemaTypeWithOptionalsWithDe final String topicName = topicName("server1", "schema", tableName); // Create record, optionals provided. - final SinkRecord createRecord = factory.createBuilder() + final KafkaDebeziumSinkRecord createRecord = factory.createBuilder() .name("prefix") .topic(topicName) .keySchema(factory.basicKeySchema()) @@ -423,7 +423,7 @@ public void shouldCreateTableWithDefaultValues(SinkRecordFactory factory) { final String topicName = topicName("server1", "schema", tableName); // Create record, optionals provided. - final SinkRecord createRecord = factory.createBuilder() + final KafkaDebeziumSinkRecord createRecord = factory.createBuilder() .name("prefix") .topic(topicName) .keySchema(factory.basicKeySchema()) diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkTest.java index a3ec49a6677..0f40e60decc 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/AbstractJdbcSinkTest.java @@ -23,6 +23,7 @@ import com.mchange.v2.c3p0.DataSources; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnector; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkTaskTestContext; @@ -151,7 +152,7 @@ protected void stopSinkConnector() { /** * Consumes the provided {@link SinkRecord} by the JDBC sink connector task. */ - protected void consume(SinkRecord record) { + protected void consume(KafkaDebeziumSinkRecord record) { if (record != null) { consume(Collections.singletonList(record)); } @@ -160,8 +161,9 @@ protected void consume(SinkRecord record) { /** * Consumes the provided collection of {@link SinkRecord} by the JDBC sink connector task. */ - protected void consume(List records) { - sinkTask.put(records); + protected void consume(List records) { + List kafkaRecords = records.stream().map(KafkaDebeziumSinkRecord::getOriginalKafkaRecord).toList(); + sinkTask.put(kafkaRecords); } /** @@ -171,10 +173,10 @@ protected String randomTableName() { return randomTableNameGenerator.randomName(); } - protected String destinationTableName(SinkRecord record) { + protected String destinationTableName(KafkaDebeziumSinkRecord record) { // todo: pass the configuration in from the test final JdbcSinkConnectorConfig config = new JdbcSinkConnectorConfig(getDefaultSinkConfig()); - return sink.formatTableName(collectionNamingStrategy.resolveCollectionName(config, record)); + return sink.formatTableName(collectionNamingStrategy.resolveCollectionName(record, config.getCollectionNameFormat())); } /** diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkColumnTypeMappingIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkColumnTypeMappingIT.java index 8ebd2cdc6aa..22dcaa50fe1 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkColumnTypeMappingIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkColumnTypeMappingIT.java @@ -11,12 +11,12 @@ import java.util.Map; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest; import io.debezium.connector.jdbc.junit.jupiter.MySqlSinkDatabaseContextProvider; @@ -59,7 +59,7 @@ public void testShouldCoerceNioByteBufferTypeToByteArrayColumnType(SinkRecordFac buffer.put((byte) 2); buffer.put((byte) 3); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkInsertModeIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkInsertModeIT.java index d986d9290c4..225b92fbd29 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkInsertModeIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkInsertModeIT.java @@ -16,7 +16,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.junit.jupiter.api.Tag; @@ -24,6 +23,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.InsertMode; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; @@ -83,7 +83,7 @@ public void testInsertModeInsertWithPrimaryKeyModeComplexRecordValue(SinkRecordF .put("wkb", Base64.getDecoder().decode("AQEAAAAAAAAAAADwPwAAAAAAAPA/".getBytes())) .put("srid", 3187); - final SinkRecord createGeometryRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, + final KafkaDebeziumSinkRecord createGeometryRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, List.of("geometry", "point", "g"), List.of(geometrySchema, pointSchema, geometrySchema), Arrays.asList(new Object[]{ geometryValue, pointValue, null })); consume(createGeometryRecord); @@ -121,7 +121,7 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord recordA = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordA = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "12345")), List.of( @@ -134,14 +134,14 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 0); - final SinkRecord recordB = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordB = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "23456")), List.of(new SchemaAndValueField("gis_area", Geometry.schema(), null), new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 1); - final SinkRecord recordC = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordC = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "23456")), List.of( @@ -154,7 +154,7 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 0); - final List records = List.of(recordA, recordB, recordC); + final List records = List.of(recordA, recordB, recordC); consume(records); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(recordA)); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkRetryIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkRetryIT.java index 05ca7383dda..dcdb6a05e37 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkRetryIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/mysql/JdbcSinkRetryIT.java @@ -16,13 +16,13 @@ import java.util.Map; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.sink.SinkRecord; import org.hibernate.PessimisticLockException; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest; import io.debezium.connector.jdbc.junit.jupiter.MySqlSinkDatabaseContextProvider; @@ -88,7 +88,7 @@ public void testRetryToFlushBufferWhenRetriableExceptionOccurred(SinkRecordFacto final String topicName = topicName("server1", "schema", tableName); - final SinkRecord updateRecord = factory.updateRecordWithSchemaValue( + final KafkaDebeziumSinkRecord updateRecord = factory.updateRecordWithSchemaValue( topicName, (byte) 1, "content", diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/oracle/JdbcSinkColumnTypeMappingIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/oracle/JdbcSinkColumnTypeMappingIT.java index 11784725106..226dfb7dacd 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/oracle/JdbcSinkColumnTypeMappingIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/oracle/JdbcSinkColumnTypeMappingIT.java @@ -11,12 +11,12 @@ import java.util.Map; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest; import io.debezium.connector.jdbc.junit.jupiter.OracleSinkDatabaseContextProvider; @@ -59,7 +59,7 @@ public void testShouldCoerceNioByteBufferTypeToByteArrayColumnType(SinkRecordFac buffer.put((byte) 2); buffer.put((byte) 3); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkColumnTypeMappingIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkColumnTypeMappingIT.java index 8e980196633..e63d526974f 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkColumnTypeMappingIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkColumnTypeMappingIT.java @@ -16,12 +16,12 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest; import io.debezium.connector.jdbc.junit.jupiter.PostgresSinkDatabaseContextProvider; @@ -81,7 +81,7 @@ private void shouldCoerceStringTypeToColumnType(SinkRecordFactory factory, Strin final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -94,7 +94,7 @@ private void shouldCoerceStringTypeToColumnType(SinkRecordFactory factory, Strin consume(createRecord); - final SinkRecord updateRecord = factory.updateRecordWithSchemaValue( + final KafkaDebeziumSinkRecord updateRecord = factory.updateRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -125,7 +125,7 @@ public void testShouldCoerceNioByteBufferTypeToByteArrayColumnType(SinkRecordFac buffer.put((byte) 2); buffer.put((byte) 3); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -159,7 +159,7 @@ public void testShouldWorkWithTextArrayWithASingleValue(SinkRecordFactory factor final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -193,7 +193,7 @@ public void testShouldWorkWithTextArray(SinkRecordFactory factory) throws Except final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -227,7 +227,7 @@ public void testShouldWorkWithTextArrayWithNullValues(SinkRecordFactory factory) final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -261,7 +261,7 @@ public void testShouldWorkWithNullTextArray(SinkRecordFactory factory) throws Ex final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -296,7 +296,7 @@ public void testShouldWorkWithEmptyArray(SinkRecordFactory factory) throws Excep final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -330,7 +330,7 @@ public void testShouldWorkWithCharacterVaryingArray(SinkRecordFactory factory) t final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -364,7 +364,7 @@ public void testShouldWorkWithIntArray(SinkRecordFactory factory) throws Excepti final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -398,7 +398,7 @@ public void testShouldWorkWithBoolArray(SinkRecordFactory factory) throws Except final String tableName = randomTableName(); final String topicName = topicName("server2", "schema", tableName); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, "data", @@ -433,7 +433,7 @@ public void testShouldWorkWithMultipleArraysWithDifferentTypes(SinkRecordFactory final String topicName = topicName("server2", "schema", tableName); final List uuids = List.of(UUID.randomUUID(), UUID.randomUUID()); - final SinkRecord createRecord = factory.createRecordWithSchemaValue( + final KafkaDebeziumSinkRecord createRecord = factory.createRecordWithSchemaValue( topicName, (byte) 1, List.of("text_data", "uuid_data"), diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkFieldFilterIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkFieldFilterIT.java index 5b24e4b62c7..f3d5879d585 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkFieldFilterIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkFieldFilterIT.java @@ -7,7 +7,6 @@ import java.util.Map; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.ValueType; import org.junit.jupiter.api.Tag; @@ -15,6 +14,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest; import io.debezium.connector.jdbc.junit.TestHelper; @@ -54,7 +54,7 @@ public void testFieldIncludeListWithInsertMode(SinkRecordFactory factory) throws startSinkConnector(properties); assertSinkConnectorIsRunning(); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(createRecord)); @@ -78,7 +78,7 @@ public void testFieldExcludeListWithInsertMode(SinkRecordFactory factory) throws startSinkConnector(properties); assertSinkConnectorIsRunning(); - final SinkRecord createRecord = factory.createRecordNoKey(topicName); + final KafkaDebeziumSinkRecord createRecord = factory.createRecordNoKey(topicName); consume(createRecord); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(createRecord)); @@ -105,7 +105,7 @@ public void testFieldIncludeListWithUpsertMode(SinkRecordFactory factory) throws startSinkConnector(properties); assertSinkConnectorIsRunning(); - final SinkRecord createRecord = factory.createRecord(topicName, (byte) 1); + final KafkaDebeziumSinkRecord createRecord = factory.createRecord(topicName, (byte) 1); consume(createRecord); consume(factory.createRecord(topicName, (byte) 1)); @@ -115,7 +115,7 @@ public void testFieldIncludeListWithUpsertMode(SinkRecordFactory factory) throws getSink().assertColumnType(tableAssert, "id", ValueType.NUMBER, (byte) 1); getSink().assertColumnType(tableAssert, "name", ValueType.TEXT, "John Doe"); - final SinkRecord updateRecord = factory.updateRecord(topicName); + final KafkaDebeziumSinkRecord updateRecord = factory.updateRecord(topicName); consume(updateRecord); final TableAssert tableAssertForUpdate = TestHelper.assertTable(dataSource(), destinationTableName(updateRecord)); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkInsertModeIT.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkInsertModeIT.java index 3eaadc3e6c3..5d248064b6a 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkInsertModeIT.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/integration/postgres/JdbcSinkInsertModeIT.java @@ -15,7 +15,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.sink.SinkRecord; import org.assertj.db.api.TableAssert; import org.assertj.db.type.DataSourceWithLetterCase; import org.assertj.db.type.ValueType; @@ -30,6 +29,7 @@ import org.postgresql.geometric.PGpoint; import org.postgresql.util.PGobject; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.InsertMode; import io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode; @@ -101,7 +101,7 @@ public void testInsertModeInsertWithPrimaryKeyModeComplexRecordValue(SinkRecordF .put("wkb", Base64.getDecoder().decode("AQUAACDmEAAAAQAAAAECAAAAAgAAAKd5xyk6JGVAC0YldQJaRsDGbTSAt/xkQMPTK2UZUkbA".getBytes())) .put("srid", 4326); - final SinkRecord createGeometryRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, + final KafkaDebeziumSinkRecord createGeometryRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, List.of("geometry", "point", "geography", "p"), List.of(geometrySchema, pointSchema, geographySchema, pointSchema), Arrays.asList(new Object[]{ geometryValue, pointValue, geographyValue })); consume(createGeometryRecord); @@ -149,7 +149,7 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord recordA = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordA = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "12345")), List.of( @@ -162,14 +162,14 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 0); - final SinkRecord recordB = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordB = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "23456")), List.of(new SchemaAndValueField("gis_area", Geometry.schema(), null), new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 1); - final SinkRecord recordC = factory.createInsertSchemaAndValue( + final KafkaDebeziumSinkRecord recordC = factory.createInsertSchemaAndValue( topicName, List.of(new SchemaAndValueField("id", Schema.STRING_SCHEMA, "23456")), List.of( @@ -182,7 +182,7 @@ public void testBatchWithDifferingSqlParameterBindings(SinkRecordFactory factory new SchemaAndValueField("__deleted", Schema.BOOLEAN_SCHEMA, false)), 0); - final List records = List.of(recordA, recordB, recordC); + final List records = List.of(recordA, recordB, recordC); consume(records); final TableAssert tableAssert = TestHelper.assertTable(dataSource(), destinationTableName(recordA)); @@ -207,8 +207,8 @@ public void testInsertModeInsertWithPrimaryKeyModeUpperCaseColumnNameWithQuotedI final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createSimpleRecord1 = factory.createRecord(topicName, (byte) 1, String::toUpperCase); - final SinkRecord createSimpleRecord2 = factory.createRecord(topicName, (byte) 2, String::toUpperCase); + final KafkaDebeziumSinkRecord createSimpleRecord1 = factory.createRecord(topicName, (byte) 1, String::toUpperCase); + final KafkaDebeziumSinkRecord createSimpleRecord2 = factory.createRecord(topicName, (byte) 2, String::toUpperCase); consume(createSimpleRecord1); consume(createSimpleRecord2); @@ -238,8 +238,8 @@ public void testInsertModeInsertWithPrimaryKeyModeUpperCaseColumnNameWithoutQuot final String tableName = randomTableName(); final String topicName = topicName("server1", "schema", tableName); - final SinkRecord createSimpleRecord1 = factory.createRecord(topicName, (byte) 1, String::toUpperCase); - final SinkRecord createSimpleRecord2 = factory.createRecord(topicName, (byte) 2, String::toUpperCase); + final KafkaDebeziumSinkRecord createSimpleRecord1 = factory.createRecord(topicName, (byte) 1, String::toUpperCase); + final KafkaDebeziumSinkRecord createSimpleRecord2 = factory.createRecord(topicName, (byte) 2, String::toUpperCase); consume(createSimpleRecord1); consume(createSimpleRecord2); @@ -276,7 +276,7 @@ public void testInsertModeInsertInfinityValues(SinkRecordFactory factory) throws Schema rangeSchema = SchemaBuilder.string().build(); - final SinkRecord createInfinityRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, + final KafkaDebeziumSinkRecord createInfinityRecord = factory.createRecordWithSchemaValue(topicName, (byte) 1, List.of("timestamp_infinity-", "timestamp_infinity+", "range_with_infinity"), List.of(zonedTimestampSchema, zonedTimestampSchema, rangeSchema), Arrays.asList(new Object[]{ "-infinity", "infinity", "[2010-01-01 14:30, infinity)" })); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/transforms/ConvertCloudEventToSaveableFormTest.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/transforms/ConvertCloudEventToSaveableFormTest.java index 79941c0263b..78df3335c77 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/transforms/ConvertCloudEventToSaveableFormTest.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/transforms/ConvertCloudEventToSaveableFormTest.java @@ -55,7 +55,7 @@ void testConvertNotCloudEventRecord(String serializerType) { final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord createRecord = factory.createRecord("test.topic"); + final SinkRecord createRecord = factory.createRecord("test.topic").getOriginalKafkaRecord(); assertThat(createRecord.valueSchema().name()).doesNotEndWith(".CloudEvents.Envelope"); final SinkRecord convertedRecord = transform.apply(createRecord); @@ -74,7 +74,7 @@ void testConvertCloudEventRecordWithEmptyMapping(String serializerType) { final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null).getOriginalKafkaRecord(); if (serializerType.equals("avro")) { assertThat(cloudEventRecord.valueSchema().name()).endsWith(".CloudEvents.Envelope"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); @@ -98,7 +98,7 @@ void testConvertCloudEventRecordWithMappingOfIdField(String serializerType) { final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null).getOriginalKafkaRecord(); if (serializerType.equals("avro")) { assertThat(cloudEventRecord.valueSchema().name()).endsWith(".CloudEvents.Envelope"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); @@ -130,7 +130,8 @@ void testConvertCloudEventRecordWithNotConfiguredCloudEventsSchemaCustomNameAndM final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), "TestCESchemaCustomName"); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), "TestCESchemaCustomName") + .getOriginalKafkaRecord(); assertThat(cloudEventRecord.valueSchema().name()).isEqualTo("TestCESchemaCustomName"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); assertThat(cloudEventRecord.valueSchema().field("id").schema()).isEqualTo(Schema.STRING_SCHEMA); @@ -156,7 +157,7 @@ void testConvertCloudEventRecordWithMisconfiguredCloudEventsSchemaCustomNameAndM final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), null); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), null).getOriginalKafkaRecord(); assertThat(cloudEventRecord.valueSchema().name()).isEqualTo("test.test.CloudEvents.Envelope"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); assertThat(cloudEventRecord.valueSchema().field("id").schema()).isEqualTo(Schema.STRING_SCHEMA); @@ -182,7 +183,8 @@ void testConvertCloudEventRecordWithConfiguredCloudEventsSchemaCustomNameAndMapp final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), "TestCESchemaCustomName"); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName("avro"), "TestCESchemaCustomName") + .getOriginalKafkaRecord(); assertThat(cloudEventRecord.valueSchema().name()).isEqualTo("TestCESchemaCustomName"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); assertThat(cloudEventRecord.valueSchema().field("id").schema()).isEqualTo(Schema.STRING_SCHEMA); @@ -210,7 +212,7 @@ void testConvertCloudEventRecordWithMappingOfDataField(String serializerType) { final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null).getOriginalKafkaRecord(); if (serializerType.equals("avro")) { assertThat(cloudEventRecord.valueSchema().name()).endsWith(".CloudEvents.Envelope"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); @@ -242,7 +244,7 @@ void testConvertCloudEventRecordWithMappingOfAllFieldsWithCustomNames(String ser final SinkRecordFactory factory = new DebeziumSinkRecordFactory(); - final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null); + final SinkRecord cloudEventRecord = factory.cloudEventRecord("test.topic", SerializerType.withName(serializerType), null).getOriginalKafkaRecord(); if (serializerType.equals("avro")) { assertThat(cloudEventRecord.valueSchema().name()).endsWith(".CloudEvents.Envelope"); assertThat(cloudEventRecord.valueSchema().fields().size()).isEqualTo(7); diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordBuilder.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordBuilder.java index a4294037835..4095b4d12bb 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordBuilder.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordBuilder.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.converters.spi.CloudEventsMaker; import io.debezium.converters.spi.SerializerType; import io.debezium.data.Envelope; @@ -161,25 +162,18 @@ public SinkRecordTypeBuilder cloudEventsSchemaName(String cloudEventsSchemaName) return this; } - public SinkRecord build() { - switch (type) { - case CREATE: - return buildCreateSinkRecord(); - case UPDATE: - return buildUpdateSinkRecord(); - case DELETE: - return buildDeleteSinkRecord(); - case TOMBSTONE: - return buildTombstoneSinkRecord(); - case TRUNCATE: - return buildTruncateSinkRecord(); - case CLOUD_EVENT: - return buildCloudEventRecord(); - } - return null; + public KafkaDebeziumSinkRecord build() { + return switch (type) { + case CREATE -> buildCreateSinkRecord(); + case UPDATE -> buildUpdateSinkRecord(); + case DELETE -> buildDeleteSinkRecord(); + case TOMBSTONE -> buildTombstoneSinkRecord(); + case TRUNCATE -> buildTruncateSinkRecord(); + case CLOUD_EVENT -> buildCloudEventRecord(); + }; } - private SinkRecord buildCreateSinkRecord() { + private KafkaDebeziumSinkRecord buildCreateSinkRecord() { Objects.requireNonNull(recordSchema, "A record schema must be provided."); Objects.requireNonNull(sourceSchema, "A source schema must be provided."); @@ -190,14 +184,14 @@ private SinkRecord buildCreateSinkRecord() { if (!flat) { final Envelope envelope = createEnvelope(); final Struct payload = envelope.create(after, source, Instant.now()); - return new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset)); } else { - return new SinkRecord(topicName, partition, keySchema, key, recordSchema, after, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, recordSchema, after, offset)); } } - private SinkRecord buildUpdateSinkRecord() { + private KafkaDebeziumSinkRecord buildUpdateSinkRecord() { Objects.requireNonNull(recordSchema, "A record schema must be provided."); Objects.requireNonNull(sourceSchema, "A source schema must be provided."); @@ -209,14 +203,14 @@ private SinkRecord buildUpdateSinkRecord() { if (!flat) { final Envelope envelope = createEnvelope(); final Struct payload = envelope.update(before, after, source, Instant.now()); - return new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset)); } else { - return new SinkRecord(topicName, partition, keySchema, key, recordSchema, after, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, recordSchema, after, offset)); } } - private SinkRecord buildDeleteSinkRecord() { + private KafkaDebeziumSinkRecord buildDeleteSinkRecord() { Objects.requireNonNull(recordSchema, "A record schema must be provided."); Objects.requireNonNull(sourceSchema, "A source schema must be provided."); @@ -227,31 +221,31 @@ private SinkRecord buildDeleteSinkRecord() { if (!flat) { final Envelope envelope = createEnvelope(); final Struct payload = envelope.delete(before, source, Instant.now()); - return new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, envelope.schema(), payload, offset)); } else { - return new SinkRecord(topicName, partition, keySchema, key, recordSchema, null, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, recordSchema, null, offset)); } } - private SinkRecord buildTombstoneSinkRecord() { + private KafkaDebeziumSinkRecord buildTombstoneSinkRecord() { final Struct key = populateStructForKey(); - return new SinkRecord(topicName, partition, keySchema, key, null, null, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, keySchema, key, null, null, offset)); } - private SinkRecord buildTruncateSinkRecord() { + private KafkaDebeziumSinkRecord buildTruncateSinkRecord() { if (!flat) { final Struct source = populateStructFromMap(new Struct(sourceSchema), sourceValues); final Envelope envelope = createEnvelope(); final Struct payload = envelope.truncate(source, Instant.now()); - return new SinkRecord(topicName, partition, null, null, envelope.schema(), payload, offset); + return new KafkaDebeziumSinkRecord(new SinkRecord(topicName, partition, null, null, envelope.schema(), payload, offset)); } else { return null; } } - private SinkRecord buildCloudEventRecord() { + private KafkaDebeziumSinkRecord buildCloudEventRecord() { final String schemaName = cloudEventsSchemaName != null ? cloudEventsSchemaName : "test.test.CloudEvents.Envelope"; final SchemaBuilder schemaBuilder = SchemaBuilder.struct() .name(schemaName) @@ -283,9 +277,9 @@ private SinkRecord buildCloudEventRecord() { ceValue = ceValueStruct; } - return new SinkRecord(baseRecord.topic(), baseRecord.kafkaPartition(), baseRecord.keySchema(), baseRecord.key(), + return new KafkaDebeziumSinkRecord(new SinkRecord(baseRecord.topic(), baseRecord.kafkaPartition(), baseRecord.keySchema(), baseRecord.key(), ceSchema, ceValue, - baseRecord.kafkaOffset(), baseRecord.timestamp(), baseRecord.timestampType(), baseRecord.headers()); + baseRecord.kafkaOffset(), baseRecord.timestamp(), baseRecord.timestampType(), baseRecord.headers())); } private Envelope createEnvelope() { diff --git a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordFactory.java b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordFactory.java index 1f318aba25e..ec62986b22f 100644 --- a/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordFactory.java +++ b/debezium-connector-jdbc/src/test/java/io/debezium/connector/jdbc/util/SinkRecordFactory.java @@ -12,8 +12,8 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.sink.SinkRecord; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.converters.spi.SerializerType; import io.debezium.data.SchemaAndValueField; @@ -199,7 +199,7 @@ default Schema allKafkaSchemaTypesSchemaWithOptionalDefaultValues() { .build(); } - default SinkRecord createRecordNoKey(String topicName) { + default KafkaDebeziumSinkRecord createRecordNoKey(String topicName) { return SinkRecordBuilder.create() .flat(isFlattened()) .name("prefix") @@ -213,15 +213,15 @@ default SinkRecord createRecordNoKey(String topicName) { .build(); } - default SinkRecord createRecord(String topicName) { + default KafkaDebeziumSinkRecord createRecord(String topicName) { return createRecord(topicName, (byte) 1); } - default SinkRecord createRecord(String topicName, byte key) { + default KafkaDebeziumSinkRecord createRecord(String topicName, byte key) { return createRecord(topicName, key, UnaryOperator.identity()); } - default SinkRecord createRecord(String topicName, byte key, UnaryOperator columnNameTransformation) { + default KafkaDebeziumSinkRecord createRecord(String topicName, byte key, UnaryOperator columnNameTransformation) { return SinkRecordBuilder.create() .flat(isFlattened()) .name("prefix") @@ -239,7 +239,7 @@ default SinkRecord createRecord(String topicName, byte key, UnaryOperator fieldNames, List fieldSchemas, List values) { + default KafkaDebeziumSinkRecord createRecordWithSchemaValue(String topicName, byte key, List fieldNames, List fieldSchemas, List values) { SinkRecordBuilder.SinkRecordTypeBuilder basicSchemaBuilder = SinkRecordBuilder.create() .flat(isFlattened()) .name("prefix") @@ -268,8 +268,7 @@ default SinkRecord createRecordWithSchemaValue(String topicName, byte key, List< return basicSchemaBuilder.build(); } - default SinkRecord createInsertSchemaAndValue(String topicName, List keyFields, List valueFields, int offset) { - + default KafkaDebeziumSinkRecord createInsertSchemaAndValue(String topicName, List keyFields, List valueFields, int offset) { Schema keySchema = null; if (!keyFields.isEmpty()) { SchemaBuilder keySchemaBuilder = SchemaBuilder.struct(); @@ -307,7 +306,7 @@ default SinkRecord createInsertSchemaAndValue(String topicName, List batch) { } private void handleTolerableWriteException( - final List batch, + final List batch, final boolean ordered, final RuntimeException e, final boolean logErrors, @@ -142,7 +143,7 @@ private void handleTolerableWriteException( } } - private static void log(final Collection records, final RuntimeException e) { + private static void log(final Collection records, final RuntimeException e) { LOGGER.error("Failed to put into the sink the following records: {}", records, e); } } diff --git a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoDbSinkConnectorTask.java b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoDbSinkConnectorTask.java index c3a154e9ae0..5653ec0966e 100644 --- a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoDbSinkConnectorTask.java +++ b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoDbSinkConnectorTask.java @@ -21,9 +21,11 @@ import com.mongodb.client.MongoClient; import com.mongodb.internal.VisibleForTesting; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.config.Configuration; import io.debezium.connector.mongodb.connection.MongoDbConnectionContext; import io.debezium.dlq.ErrorReporter; +import io.debezium.sink.DebeziumSinkRecord; public class MongoDbSinkConnectorTask extends SinkTask { static final Logger LOGGER = LoggerFactory.getLogger(MongoDbSinkConnectorTask.class); @@ -117,7 +119,11 @@ private ErrorReporter createErrorReporter() { try { ErrantRecordReporter errantRecordReporter = context.errantRecordReporter(); if (errantRecordReporter != null) { - result = errantRecordReporter::report; + result = (DebeziumSinkRecord record, Exception e) -> { + if (record instanceof KafkaDebeziumSinkRecord kafkaRecord) { + errantRecordReporter.report(kafkaRecord.getOriginalKafkaRecord(), e); + } + }; } else { LOGGER.info("Errant record reporter not configured."); diff --git a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoProcessedSinkRecordData.java b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoProcessedSinkRecordData.java index 73eec571bd8..56543e3b782 100644 --- a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoProcessedSinkRecordData.java +++ b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoProcessedSinkRecordData.java @@ -8,7 +8,6 @@ import java.util.Optional; import java.util.function.Supplier; -import org.apache.kafka.connect.sink.SinkRecord; import org.bson.BsonDocument; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -19,19 +18,20 @@ import io.debezium.connector.mongodb.sink.converters.SinkDocument; import io.debezium.connector.mongodb.sink.converters.SinkRecordConverter; import io.debezium.connector.mongodb.sink.eventhandler.relational.RelationalEventHandler; +import io.debezium.sink.DebeziumSinkRecord; public class MongoProcessedSinkRecordData { private static final Logger LOGGER = LoggerFactory.getLogger(MongoProcessedSinkRecordData.class); private final MongoDbSinkConnectorConfig config; private final MongoNamespace namespace; - private final SinkRecord sinkRecord; + private final DebeziumSinkRecord sinkRecord; private final SinkDocument sinkDocument; private final WriteModel writeModel; private Exception exception; private final String databaseName; - MongoProcessedSinkRecordData(final SinkRecord sinkRecord, final MongoDbSinkConnectorConfig sinkConfig) { + MongoProcessedSinkRecordData(final DebeziumSinkRecord sinkRecord, final MongoDbSinkConnectorConfig sinkConfig) { this.sinkRecord = sinkRecord; this.databaseName = sinkConfig.getSinkDatabaseName(); this.config = sinkConfig; @@ -48,7 +48,7 @@ public MongoNamespace getNamespace() { return namespace; } - public SinkRecord getSinkRecord() { + public DebeziumSinkRecord getSinkRecord() { return sinkRecord; } @@ -61,9 +61,10 @@ public Exception getException() { } private MongoNamespace createNamespace() { - return tryProcess( - () -> Optional.of(new MongoNamespace(databaseName, config.getCollectionNamingStrategy().resolveCollectionName(config, sinkRecord)))) + () -> Optional.of(new MongoNamespace( + databaseName, + config.getCollectionNamingStrategy().resolveCollectionName(sinkRecord, config.getCollectionNameFormat())))) .orElse(null); } diff --git a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoSinkRecordProcessor.java b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoSinkRecordProcessor.java index 38f9dd69dff..aee3fce4cf5 100644 --- a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoSinkRecordProcessor.java +++ b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/MongoSinkRecordProcessor.java @@ -13,7 +13,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.dlq.ErrorReporter; +import io.debezium.sink.DebeziumSinkRecord; final class MongoSinkRecordProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(MongoSinkRecordProcessor.class); @@ -25,10 +27,11 @@ static List> orderedGroupByTopicAndNamespace( LOGGER.debug("Number of sink records to process: {}", records.size()); List> orderedProcessedSinkRecordData = new ArrayList<>(); - List currentGroup = new ArrayList<>(); + List groupedBatch = new ArrayList<>(); MongoProcessedSinkRecordData previous = null; - for (SinkRecord record : records) { + for (SinkRecord kafkaSinkRecord : records) { + DebeziumSinkRecord record = new KafkaDebeziumSinkRecord(kafkaSinkRecord); MongoProcessedSinkRecordData processedData = new MongoProcessedSinkRecordData(record, sinkConfig); if (processedData.getException() != null) { @@ -45,23 +48,23 @@ else if (processedData.getNamespace() == null || processedData.getWriteModel() = } int maxBatchSize = processedData.getConfig().getBatchSize(); - if (maxBatchSize > 0 && currentGroup.size() == maxBatchSize - || !previous.getSinkRecord().topic().equals(processedData.getSinkRecord().topic()) + if (maxBatchSize > 0 && groupedBatch.size() == maxBatchSize + || !previous.getSinkRecord().topicName().equals(processedData.getSinkRecord().topicName()) || !previous.getNamespace().equals(processedData.getNamespace())) { - orderedProcessedSinkRecordData.add(currentGroup); - currentGroup = new ArrayList<>(); + orderedProcessedSinkRecordData.add(groupedBatch); + groupedBatch = new ArrayList<>(); } previous = processedData; - currentGroup.add(processedData); + groupedBatch.add(processedData); } - if (!currentGroup.isEmpty()) { - orderedProcessedSinkRecordData.add(currentGroup); + if (!groupedBatch.isEmpty()) { + orderedProcessedSinkRecordData.add(groupedBatch); } return orderedProcessedSinkRecordData; } private MongoSinkRecordProcessor() { } -} \ No newline at end of file +} diff --git a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/LazyBsonDocument.java b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/LazyBsonDocument.java index 265a545a24b..12613271343 100644 --- a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/LazyBsonDocument.java +++ b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/LazyBsonDocument.java @@ -17,12 +17,13 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.bson.BsonDocument; import org.bson.BsonValue; import com.mongodb.lang.Nullable; +import io.debezium.sink.DebeziumSinkRecord; + /** * A lazy representation of a MongoDB document. */ @@ -30,7 +31,7 @@ public class LazyBsonDocument extends BsonDocument { @Serial private static final long serialVersionUID = 1L; - private final transient SinkRecord record; + private final transient DebeziumSinkRecord record; private final transient Type dataType; private final transient BiFunction converter; @@ -48,7 +49,7 @@ public enum Type { * @param converter the converter for the sink record */ public LazyBsonDocument( - final SinkRecord record, + final DebeziumSinkRecord record, final Type dataType, final BiFunction converter) { if (record == null) { diff --git a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/SinkRecordConverter.java b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/SinkRecordConverter.java index 8bb9dd18073..f063ff038cb 100644 --- a/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/SinkRecordConverter.java +++ b/debezium-connector-mongodb/src/main/java/io/debezium/connector/mongodb/sink/converters/SinkRecordConverter.java @@ -10,12 +10,12 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.bson.BsonDocument; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.debezium.connector.mongodb.sink.converters.LazyBsonDocument.Type; +import io.debezium.sink.DebeziumSinkRecord; /** * Converts a Kafka Connect record into one BsonDocument for the record key and one for the record value, choosing the correct SinkValueConverter based on the record type. @@ -28,7 +28,7 @@ public class SinkRecordConverter { private static final SinkValueConverter JSON_STRING_VALUE_CONVERTER = new JsonStringValueConverter(); private static final SinkValueConverter BYTE_ARRAY_VALUE_CONVERTER = new ByteArrayValueConverter(); - public SinkDocument convert(final SinkRecord record) { + public SinkDocument convert(final DebeziumSinkRecord record) { LOGGER.trace("record: {}", record); BsonDocument keyDoc = null; diff --git a/debezium-core/src/main/java/io/debezium/schema/SchemaFactory.java b/debezium-core/src/main/java/io/debezium/schema/SchemaFactory.java index 4b482db3621..9d46ccce0f3 100644 --- a/debezium-core/src/main/java/io/debezium/schema/SchemaFactory.java +++ b/debezium-core/src/main/java/io/debezium/schema/SchemaFactory.java @@ -69,7 +69,8 @@ public class SchemaFactory { private static final String SCHEMA_HISTORY_CONNECTOR_KEY_SCHEMA_NAME_SUFFIX = ".SchemaChangeKey"; private static final int SCHEMA_HISTORY_CONNECTOR_KEY_SCHEMA_VERSION = 1; - private static final String SCHEMA_HISTORY_CONNECTOR_VALUE_SCHEMA_NAME_SUFFIX = ".SchemaChangeValue"; + public static final String SCHEMA_CHANGE_VALUE = "SchemaChangeValue"; + private static final String SCHEMA_HISTORY_CONNECTOR_VALUE_SCHEMA_NAME_SUFFIX = "." + SCHEMA_CHANGE_VALUE; private static final int SCHEMA_HISTORY_CONNECTOR_VALUE_SCHEMA_VERSION = 1; private static final String SCHEMA_HISTORY_TABLE_SCHEMA_NAME = "io.debezium.connector.schema.Table"; diff --git a/debezium-sink/pom.xml b/debezium-sink/pom.xml index 12984f6c291..ac84fda7a0a 100644 --- a/debezium-sink/pom.xml +++ b/debezium-sink/pom.xml @@ -14,12 +14,6 @@ Debezium Sink Debezium Sink module - - 17 - 17 - UTF-8 - - io.debezium diff --git a/debezium-sink/src/main/java/io/debezium/bindings/kafka/KafkaDebeziumSinkRecord.java b/debezium-sink/src/main/java/io/debezium/bindings/kafka/KafkaDebeziumSinkRecord.java new file mode 100644 index 00000000000..04cafcda2f7 --- /dev/null +++ b/debezium-sink/src/main/java/io/debezium/bindings/kafka/KafkaDebeziumSinkRecord.java @@ -0,0 +1,162 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.bindings.kafka; + +import java.util.List; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.sink.SinkRecord; + +import io.debezium.annotation.Immutable; +import io.debezium.data.Envelope; +import io.debezium.sink.DebeziumSinkRecord; +import io.debezium.sink.SinkConnectorConfig; +import io.debezium.util.Strings; + +@Immutable +public class KafkaDebeziumSinkRecord implements DebeziumSinkRecord { + + protected final SinkRecord originalKafkaRecord; + + public KafkaDebeziumSinkRecord(SinkRecord record) { + this.originalKafkaRecord = record; + } + + @Override + public String topicName() { + return originalKafkaRecord.topic(); + } + + @Override + public Integer partition() { + return originalKafkaRecord.kafkaPartition(); + } + + @Override + public long offset() { + return originalKafkaRecord.kafkaOffset(); + } + + @Override + public List keyFieldNames() { + throw new RuntimeException("Not implemented"); + } + + @Override + public Object key() { + return originalKafkaRecord.key(); + } + + @Override + public Schema keySchema() { + return originalKafkaRecord.keySchema(); + } + + public Object value() { + return originalKafkaRecord.value(); + } + + @Override + public Schema valueSchema() { + return originalKafkaRecord.valueSchema(); + } + + @Override + public boolean isDebeziumMessage() { + return originalKafkaRecord.value() != null && originalKafkaRecord.valueSchema().name() != null && originalKafkaRecord.valueSchema().name().contains("Envelope"); + } + + public boolean isSchemaChange() { + return originalKafkaRecord.valueSchema() != null + && !Strings.isNullOrEmpty(originalKafkaRecord.valueSchema().name()) + && originalKafkaRecord.valueSchema().name().contains(SCHEMA_CHANGE_VALUE); + } + + public boolean isFlattened() { + return !isTombstone() && (originalKafkaRecord.valueSchema().name() == null || !originalKafkaRecord.valueSchema().name().contains("Envelope")); + } + + @Override + public boolean isTombstone() { + // NOTE + // Debezium TOMBSTONE has both value and valueSchema to null, instead of the ExtractNewRecordState SMT with delete.handling.mode=none + // which will generate a record with value null that should be treated as a flattened delete. See isDelete method. + return originalKafkaRecord.value() == null && originalKafkaRecord.valueSchema() == null; + } + + @Override + public boolean isDelete() { + if (!isDebeziumMessage()) { + return originalKafkaRecord.value() == null; + } + else if (originalKafkaRecord.value() != null) { + final Struct value = (Struct) originalKafkaRecord.value(); + return Envelope.Operation.DELETE.equals(Envelope.Operation.forCode(value.getString(Envelope.FieldName.OPERATION))); + } + return false; + } + + @Override + public boolean isTruncate() { + if (isDebeziumMessage()) { + final Struct value = (Struct) originalKafkaRecord.value(); + return Envelope.Operation.TRUNCATE.equals(Envelope.Operation.forCode(value.getString(Envelope.FieldName.OPERATION))); + } + return false; + } + + public Struct getPayload() { + if (isDebeziumMessage()) { + return ((Struct) originalKafkaRecord.value()).getStruct(Envelope.FieldName.AFTER); + } + else { + return ((Struct) originalKafkaRecord.value()); + } + } + + @Override + public Struct getKeyStruct(SinkConnectorConfig.PrimaryKeyMode primaryKeyMode) { + if (!keyFieldNames().isEmpty()) { + switch (primaryKeyMode) { + case RECORD_KEY: + final Schema keySchema = originalKafkaRecord.keySchema(); + if (keySchema != null && Schema.Type.STRUCT.equals(keySchema.type())) { + return (Struct) originalKafkaRecord.key(); + } + else { + throw new ConnectException("No struct-based primary key defined for record key."); + } + case RECORD_VALUE: + final Schema valueSchema = originalKafkaRecord.valueSchema(); + if (valueSchema != null && Schema.Type.STRUCT.equals(valueSchema.type())) { + return getPayload(); + } + else { + throw new ConnectException("No struct-based primary key defined for record value."); + } + + case RECORD_HEADER: + final SchemaBuilder headerSchemaBuilder = SchemaBuilder.struct(); + originalKafkaRecord.headers().forEach((Header header) -> headerSchemaBuilder.field(header.key(), header.schema())); + + final Schema headerSchema = headerSchemaBuilder.build(); + final Struct headerStruct = new Struct(headerSchema); + originalKafkaRecord.headers().forEach((Header header) -> headerStruct.put(header.key(), header.value())); + return headerStruct; + } + } + return null; + } + + public SinkRecord getOriginalKafkaRecord() { + return originalKafkaRecord; + } + +} diff --git a/debezium-core/src/main/java/io/debezium/dlq/ErrorReporter.java b/debezium-sink/src/main/java/io/debezium/dlq/ErrorReporter.java similarity index 68% rename from debezium-core/src/main/java/io/debezium/dlq/ErrorReporter.java rename to debezium-sink/src/main/java/io/debezium/dlq/ErrorReporter.java index e53bd4c5ed2..316b2a78757 100644 --- a/debezium-core/src/main/java/io/debezium/dlq/ErrorReporter.java +++ b/debezium-sink/src/main/java/io/debezium/dlq/ErrorReporter.java @@ -5,8 +5,8 @@ */ package io.debezium.dlq; -import org.apache.kafka.connect.sink.SinkRecord; +import io.debezium.sink.DebeziumSinkRecord; public interface ErrorReporter { - void report(SinkRecord record, Exception e); + void report(DebeziumSinkRecord record, Exception e); } diff --git a/debezium-sink/src/main/java/io/debezium/sink/DebeziumSinkRecord.java b/debezium-sink/src/main/java/io/debezium/sink/DebeziumSinkRecord.java new file mode 100644 index 00000000000..f380a5b78a0 --- /dev/null +++ b/debezium-sink/src/main/java/io/debezium/sink/DebeziumSinkRecord.java @@ -0,0 +1,52 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.sink; + +import java.util.List; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; // @TODO find a good replacement for Kafka Connect's Struct and Schema + +import io.debezium.annotation.Immutable; +import io.debezium.schema.SchemaFactory; +import io.debezium.sink.SinkConnectorConfig.PrimaryKeyMode; + +@Immutable +public interface DebeziumSinkRecord { + + String SCHEMA_CHANGE_VALUE = SchemaFactory.SCHEMA_CHANGE_VALUE; + + String topicName(); + + Integer partition(); + + long offset(); + + Object key(); + + Schema keySchema(); + + Object value(); + + Schema valueSchema(); + + boolean isDebeziumMessage(); + + boolean isSchemaChange(); + + boolean isTombstone(); + + boolean isDelete(); + + boolean isTruncate(); + + Struct getPayload(); + + List keyFieldNames(); + + Struct getKeyStruct(PrimaryKeyMode primaryKeyMode); + +} diff --git a/debezium-sink/src/main/java/io/debezium/sink/naming/CollectionNamingStrategy.java b/debezium-sink/src/main/java/io/debezium/sink/naming/CollectionNamingStrategy.java index 3face35364a..02da8a966b6 100644 --- a/debezium-sink/src/main/java/io/debezium/sink/naming/CollectionNamingStrategy.java +++ b/debezium-sink/src/main/java/io/debezium/sink/naming/CollectionNamingStrategy.java @@ -5,9 +5,7 @@ */ package io.debezium.sink.naming; -import org.apache.kafka.connect.sink.SinkRecord; - -import io.debezium.sink.SinkConnectorConfig; +import io.debezium.sink.DebeziumSinkRecord; /** * A pluggable strategy contract for defining how table names are resolved from kafka records. @@ -18,9 +16,9 @@ public interface CollectionNamingStrategy { /** * Resolves the logical table name from the sink record. * - * @param config the sink connector configuration defining table name format * @param record Debezium sink record, should not be {@code null} + * @param collectionNameFormat the format string for the collection name (mapped from the topic name) * @return the resolved logical table name; if {@code null} the record should not be processed */ - String resolveCollectionName(SinkConnectorConfig config, SinkRecord record); + String resolveCollectionName(DebeziumSinkRecord record, String collectionNameFormat); } diff --git a/debezium-sink/src/main/java/io/debezium/sink/naming/DefaultCollectionNamingStrategy.java b/debezium-sink/src/main/java/io/debezium/sink/naming/DefaultCollectionNamingStrategy.java index 7da3160bc04..796e4b145b6 100644 --- a/debezium-sink/src/main/java/io/debezium/sink/naming/DefaultCollectionNamingStrategy.java +++ b/debezium-sink/src/main/java/io/debezium/sink/naming/DefaultCollectionNamingStrategy.java @@ -10,12 +10,10 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.debezium.data.Envelope; -import io.debezium.sink.SinkConnectorConfig; +import io.debezium.sink.DebeziumSinkRecord; /** * Default implementation of the {@link CollectionNamingStrategy} where the table name is driven @@ -28,46 +26,44 @@ public class DefaultCollectionNamingStrategy implements CollectionNamingStrategy { private static final Logger LOGGER = LoggerFactory.getLogger(DefaultCollectionNamingStrategy.class); + private static final String ENVELOPE_SOURCE_FIELD_NAME = "source"; + private final Pattern sourcePattern = Pattern.compile("\\$\\{(source\\.)(.*?)}"); @Override - public String resolveCollectionName(SinkConnectorConfig config, SinkRecord record) { + public String resolveCollectionName(DebeziumSinkRecord record, String collectionNameFormat) { // Default behavior is to replace dots with underscores - final String topicName = record.topic().replace(".", "_"); - String table = config.getCollectionNameFormat().replace("${topic}", topicName); + final String topicName = record.topicName().replace(".", "_"); + String collection = collectionNameFormat.replace("${topic}", topicName); - table = resolveCollectionNameBySource(config, record, table); - return table; + collection = resolveCollectionNameBySource(record, collection, collectionNameFormat); + return collection; } - private String resolveCollectionNameBySource(SinkConnectorConfig config, SinkRecord record, String tableFormat) { - String table = tableFormat; - if (table.contains("${source.")) { - if (isTombstone(record)) { + private String resolveCollectionNameBySource(DebeziumSinkRecord record, String collectionName, String collectionNameFormat) { + if (collectionName.contains("${source.")) { + if (!record.isDebeziumMessage()) { LOGGER.warn( - "Ignore this record because it seems to be a tombstone that doesn't have source field, then cannot resolve table name in topic '{}', partition '{}', offset '{}'", - record.topic(), record.kafkaPartition(), record.kafkaOffset()); + "Ignore this record because it isn't a Debezium record, then cannot resolve a collection name in topic '{}', partition '{}', offset '{}'", + record.topicName(), record.partition(), record.offset()); return null; } try { - Struct source = ((Struct) record.value()).getStruct(Envelope.FieldName.SOURCE); - Matcher matcher = sourcePattern.matcher(table); + Struct source = ((Struct) record.value()).getStruct(ENVELOPE_SOURCE_FIELD_NAME); + Matcher matcher = sourcePattern.matcher(collectionName); while (matcher.find()) { String target = matcher.group(); - table = table.replace(target, source.getString(matcher.group(2))); + collectionName = collectionName.replace(target, source.getString(matcher.group(2))); } } catch (DataException e) { - LOGGER.error("Failed to resolve table name with format '{}', check source field in topic '{}'", config.getCollectionNameFormat(), record.topic(), e); + LOGGER.error("Failed to resolve collection name with format '{}', check source field in topic '{}'", + collectionNameFormat, record.topicName(), e); throw e; } } - return table; - } - - private boolean isTombstone(SinkRecord record) { - return record.value() == null; + return collectionName; } } diff --git a/debezium-testing/debezium-testing-system/src/test/java/io/debezium/testing/system/tests/jdbc/sink/JdbcSinkTests.java b/debezium-testing/debezium-testing-system/src/test/java/io/debezium/testing/system/tests/jdbc/sink/JdbcSinkTests.java index bfe79e8def2..1a2236e8b93 100644 --- a/debezium-testing/debezium-testing-system/src/test/java/io/debezium/testing/system/tests/jdbc/sink/JdbcSinkTests.java +++ b/debezium-testing/debezium-testing-system/src/test/java/io/debezium/testing/system/tests/jdbc/sink/JdbcSinkTests.java @@ -19,12 +19,12 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.json.JsonConverter; -import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import io.debezium.bindings.kafka.KafkaDebeziumSinkRecord; import io.debezium.connector.jdbc.util.DebeziumSinkRecordFactory; import io.debezium.connector.jdbc.util.SinkRecordBuilder; import io.debezium.testing.system.assertions.JdbcAssertions; @@ -66,7 +66,7 @@ private void produceRecordToTopic(String topic, String fieldName, String fieldVa private String createRecord(String fieldName, String fieldValue) { DebeziumSinkRecordFactory factory = new DebeziumSinkRecordFactory(); - SinkRecord record = SinkRecordBuilder.update() // TODO: Change to create when fixed in JDBC connector testsuite + KafkaDebeziumSinkRecord record = SinkRecordBuilder.update() // TODO: Change to create when fixed in JDBC connector testsuite .flat(false) .name("jdbc-connector-test") .recordSchema(SchemaBuilder.struct().field(fieldName, Schema.STRING_SCHEMA).build())