Skip to content
This repository has been archived by the owner on Sep 13, 2024. It is now read-only.

DBZ-7620 JDBC connector does not process ByteBuffer field value #66

Merged
merged 2 commits into from
Mar 15, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -8,23 +8,18 @@
import org.apache.kafka.connect.data.Schema;

import io.debezium.connector.jdbc.dialect.DatabaseDialect;
import io.debezium.connector.jdbc.type.AbstractType;
import io.debezium.connector.jdbc.type.AbstractBytesType;
import io.debezium.connector.jdbc.type.Type;

/**
* An implementation of {@link Type} for {@code BYTES} column types.
*
* @author Chris Cranford
*/
class BytesType extends AbstractType {
class BytesType extends AbstractBytesType {

public static final BytesType INSTANCE = new BytesType();

@Override
public String[] getRegistrationKeys() {
return new String[]{ "BYTES" };
}

@Override
public String getTypeName(DatabaseDialect dialect, Schema schema, boolean key) {
// Hibernate defaults to VARCHAR(n) FOR BIT DATA when using Types.VARBINARY.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,23 +12,18 @@
import org.hibernate.engine.jdbc.Size;

import io.debezium.connector.jdbc.dialect.DatabaseDialect;
import io.debezium.connector.jdbc.type.AbstractType;
import io.debezium.connector.jdbc.type.AbstractBytesType;
import io.debezium.connector.jdbc.type.Type;

/**
* An implementation of {@link Type} for {@code BYTES} column types.
*
* @author Chris Cranford
*/
class BytesType extends AbstractType {
class BytesType extends AbstractBytesType {

public static final BytesType INSTANCE = new BytesType();

@Override
public String[] getRegistrationKeys() {
return new String[]{ "BYTES" };
}

@Override
public String getTypeName(DatabaseDialect dialect, Schema schema, boolean key) {
final Optional<String> columnType = getSourceColumnType(schema);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,23 +8,18 @@
import org.apache.kafka.connect.data.Schema;

import io.debezium.connector.jdbc.dialect.DatabaseDialect;
import io.debezium.connector.jdbc.type.AbstractType;
import io.debezium.connector.jdbc.type.AbstractBytesType;
import io.debezium.connector.jdbc.type.Type;

/**
* An implementation of {@link Type} for {@code BYTES} column types.
*
* @author Chris Cranford
*/
class BytesType extends AbstractType {
class BytesType extends AbstractBytesType {

public static final BytesType INSTANCE = new BytesType();

@Override
public String[] getRegistrationKeys() {
return new String[]{ "BYTES" };
}

@Override
public String getTypeName(DatabaseDialect dialect, Schema schema, boolean key) {
// Hibernate will elect to use RAW(n) when column propagation is enabled, and we ideally do not want
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,16 +5,13 @@
*/
package io.debezium.connector.jdbc.dialect.postgres;

import java.nio.ByteBuffer;
import java.sql.Types;
import java.util.List;

import org.apache.kafka.connect.data.Schema;
import org.hibernate.engine.jdbc.Size;

import io.debezium.connector.jdbc.ValueBindDescriptor;
import io.debezium.connector.jdbc.dialect.DatabaseDialect;
import io.debezium.connector.jdbc.type.AbstractType;
import io.debezium.connector.jdbc.type.AbstractBytesType;
import io.debezium.connector.jdbc.type.Type;
import io.debezium.connector.jdbc.util.ByteArrayUtils;

Expand All @@ -23,15 +20,10 @@
*
* @author Bertrand Paquet
*/
class BytesType extends AbstractType {
class BytesType extends AbstractBytesType {

public static final BytesType INSTANCE = new BytesType();

@Override
public String[] getRegistrationKeys() {
return new String[]{ "BYTES" };
}

@Override
public String getDefaultValueBinding(DatabaseDialect dialect, Schema schema, Object value) {
return String.format(dialect.getByteArrayFormat(), ByteArrayUtils.getByteArrayAsHex(value));
Expand All @@ -48,14 +40,4 @@ else if (key) {
}
return dialect.getTypeName(Types.VARBINARY);
}

@Override
public List<ValueBindDescriptor> bind(int index, Schema schema, Object value) {

if (value instanceof ByteBuffer) {
return List.of(new ValueBindDescriptor(index, ((ByteBuffer) value).array()));
}

return super.bind(index, schema, value);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.type;

import java.nio.ByteBuffer;
import java.util.List;

import org.apache.kafka.connect.data.Schema;

import io.debezium.connector.jdbc.ValueBindDescriptor;

/**
* An abstract {@link Type} implementation for {@code BYTES} column types.
*
* @author Chris Cranford
*/
public abstract class AbstractBytesType extends AbstractType {

@Override
public String[] getRegistrationKeys() {
return new String[]{ "BYTES" };
}

@Override
public List<ValueBindDescriptor> bind(int index, Schema schema, Object value) {
if (value instanceof ByteBuffer) {
return List.of(new ValueBindDescriptor(index, ((ByteBuffer) value).array()));
}
return super.bind(index, schema, value);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.integration.mysql;

import static org.fest.assertions.Assertions.assertThat;

import java.nio.ByteBuffer;
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.connector.jdbc.JdbcSinkConnectorConfig;
import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest;
import io.debezium.connector.jdbc.junit.jupiter.MySqlSinkDatabaseContextProvider;
import io.debezium.connector.jdbc.junit.jupiter.Sink;
import io.debezium.connector.jdbc.junit.jupiter.SinkRecordFactoryArgumentsProvider;
import io.debezium.connector.jdbc.util.SinkRecordFactory;
import io.debezium.doc.FixFor;

/**
* Column type mappings for MySQL
*
* @author Chris Cranford
*/
@Tag("all")
@Tag("it")
@Tag("it-mysql")
@ExtendWith(MySqlSinkDatabaseContextProvider.class)
public class JdbcSinkColumnTypeMappingIT extends AbstractJdbcSinkTest {

public JdbcSinkColumnTypeMappingIT(Sink sink) {
super(sink);
}

@ParameterizedTest
@ArgumentsSource(SinkRecordFactoryArgumentsProvider.class)
@FixFor("DBZ-6967")
public void testShouldCoerceNioByteBufferTypeToByteArrayColumnType(SinkRecordFactory factory) throws Exception {
final Map<String, String> properties = getDefaultSinkConfig();
properties.put(JdbcSinkConnectorConfig.SCHEMA_EVOLUTION, JdbcSinkConnectorConfig.SchemaEvolutionMode.NONE.getValue());
properties.put(JdbcSinkConnectorConfig.PRIMARY_KEY_MODE, JdbcSinkConnectorConfig.PrimaryKeyMode.RECORD_KEY.getValue());
properties.put(JdbcSinkConnectorConfig.INSERT_MODE, JdbcSinkConnectorConfig.InsertMode.UPSERT.getValue());
startSinkConnector(properties);
assertSinkConnectorIsRunning();

final String tableName = randomTableName();
final String topicName = topicName("server2", "schema", tableName);

ByteBuffer buffer = ByteBuffer.allocate(3);
buffer.put((byte) 1);
buffer.put((byte) 2);
buffer.put((byte) 3);

final SinkRecord createRecord = factory.createRecordWithSchemaValue(
topicName,
(byte) 1,
"data",
Schema.OPTIONAL_BYTES_SCHEMA,
buffer);

final String destinationTable = destinationTableName(createRecord);
final String sql = "CREATE TABLE %s (id int not null, data binary(3), primary key(id))";
getSink().execute(String.format(sql, destinationTable));

consume(createRecord);

getSink().assertRows(destinationTable, rs -> {
assertThat(rs.getInt(1)).isEqualTo(1);
assertThat(rs.getBytes(2)).isEqualTo(new byte[]{ 1, 2, 3 });
return null;
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.integration.oracle;

import static org.fest.assertions.Assertions.assertThat;

import java.nio.ByteBuffer;
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.connector.jdbc.JdbcSinkConnectorConfig;
import io.debezium.connector.jdbc.integration.AbstractJdbcSinkTest;
import io.debezium.connector.jdbc.junit.jupiter.OracleSinkDatabaseContextProvider;
import io.debezium.connector.jdbc.junit.jupiter.Sink;
import io.debezium.connector.jdbc.junit.jupiter.SinkRecordFactoryArgumentsProvider;
import io.debezium.connector.jdbc.util.SinkRecordFactory;
import io.debezium.doc.FixFor;

/**
* Column type mapping tests for Oracle.
*
* @author Andrey Pustovetov
*/
@Tag("all")
@Tag("it")
@Tag("it-oracle")
@ExtendWith(OracleSinkDatabaseContextProvider.class)
public class JdbcSinkColumnTypeMappingIT extends AbstractJdbcSinkTest {

public JdbcSinkColumnTypeMappingIT(Sink sink) {
super(sink);
}

@ParameterizedTest
@ArgumentsSource(SinkRecordFactoryArgumentsProvider.class)
@FixFor("DBZ-7620")
public void testShouldCoerceNioByteBufferTypeToByteArrayColumnType(SinkRecordFactory factory) throws Exception {
final Map<String, String> properties = getDefaultSinkConfig();
properties.put(JdbcSinkConnectorConfig.SCHEMA_EVOLUTION, JdbcSinkConnectorConfig.SchemaEvolutionMode.NONE.getValue());
properties.put(JdbcSinkConnectorConfig.PRIMARY_KEY_MODE, JdbcSinkConnectorConfig.PrimaryKeyMode.RECORD_KEY.getValue());
properties.put(JdbcSinkConnectorConfig.INSERT_MODE, JdbcSinkConnectorConfig.InsertMode.UPSERT.getValue());
startSinkConnector(properties);
assertSinkConnectorIsRunning();

final String tableName = randomTableName();
final String topicName = topicName("server1", "schema", tableName);

ByteBuffer buffer = ByteBuffer.allocate(3);
buffer.put((byte) 1);
buffer.put((byte) 2);
buffer.put((byte) 3);

final SinkRecord createRecord = factory.createRecordWithSchemaValue(
topicName,
(byte) 1,
"data",
Schema.OPTIONAL_BYTES_SCHEMA,
buffer);

final String destinationTable = destinationTableName(createRecord);
final String sql = "CREATE TABLE %s (id number(9,0) not null, data blob, primary key(id))";
getSink().execute(String.format(sql, destinationTable));

consume(createRecord);

getSink().assertRows(destinationTable, rs -> {
assertThat(rs.getInt(1)).isEqualTo(1);
assertThat(rs.getBytes(2)).isEqualTo(new byte[]{ 1, 2, 3 });
return null;
});
}
}