Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Added support of BSON type #63

Merged
merged 2 commits into from
Nov 19, 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
64 changes: 31 additions & 33 deletions src/main/java/com/singlestore/debezium/SingleStoreConnection.java
Original file line number Diff line number Diff line change
@@ -1,46 +1,43 @@
package com.singlestore.debezium;

import com.singlestore.debezium.util.ObserveResultSetUtils;
import static io.debezium.config.CommonConnectorConfig.DATABASE_CONFIG_PREFIX;
import static io.debezium.config.CommonConnectorConfig.DRIVER_CONFIG_PREFIX;

import com.singlestore.debezium.util.Utils;
import com.singlestore.jdbc.DatabaseMetaData;
import io.debezium.DebeziumException;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.relational.Attribute;
import io.debezium.relational.ColumnId;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.relational.Tables.ColumnNameFilter;
import io.debezium.relational.Tables.TableFilter;
import io.debezium.util.Strings;
import java.sql.Connection;
import java.sql.ResultSet;
import javax.swing.text.html.Option;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;

import static io.debezium.config.CommonConnectorConfig.DATABASE_CONFIG_PREFIX;
import static io.debezium.config.CommonConnectorConfig.DRIVER_CONFIG_PREFIX;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* {@link JdbcConnection} extension to be used with SingleStore
*/
public class SingleStoreConnection extends JdbcConnection {

protected static final String URL_PATTERN = "jdbc:singlestore://${hostname}:${port}/?connectTimeout=${connectTimeout}";
protected static final String URL_PATTERN_DATABASE = "jdbc:singlestore://${hostname}:${port}/${dbname}?connectTimeout=${connectTimeout}";
private static final Logger LOGGER = LoggerFactory.getLogger(
SingleStoreConnection.class);
private static final String QUOTED_CHARACTER = "`";
protected static final String URL_PATTERN = "jdbc:singlestore://${hostname}:${port}/?connectTimeout=${connectTimeout}";
protected static final String URL_PATTERN_DATABASE = "jdbc:singlestore://${hostname}:${port}/${dbname}?connectTimeout=${connectTimeout}";
private final SingleStoreConnectionConfiguration connectionConfig;

public SingleStoreConnection(SingleStoreConnectionConfiguration connectionConfig) {
Expand All @@ -49,6 +46,13 @@ public SingleStoreConnection(SingleStoreConnectionConfiguration connectionConfig
this.connectionConfig = connectionConfig;
}

private static void validateServerVersion(Statement statement) throws SQLException {
DatabaseMetaData metaData = (DatabaseMetaData) statement.getConnection().getMetaData();
if (!metaData.getVersion().versionGreaterOrEqual(8, 7, 16)) {
throw new SQLException("The lowest supported version of SingleStore is 8.7.16");
}
}

public String generateObserveQuery(TableId table, List<String> offsets) {
return observeQuery(null, Set.of(table), Optional.empty(), Optional.empty(),
Optional.of(String.format("(%s)", offsets
Expand All @@ -57,13 +61,6 @@ public String generateObserveQuery(TableId table, List<String> offsets) {
.collect(Collectors.joining(",")))), Optional.empty());
}

private static void validateServerVersion(Statement statement) throws SQLException {
DatabaseMetaData metaData = (DatabaseMetaData) statement.getConnection().getMetaData();
if (!metaData.getVersion().versionGreaterOrEqual(8, 7, 16)) {
throw new SQLException("The lowest supported version of SingleStore is 8.7.16");
}
}

/**
* Executes OBSERVE query for CDC output stream events.
*
Expand Down Expand Up @@ -235,7 +232,7 @@ public String quotedColumnIdString(ColumnId columnId) {
char quotingChar = '`';
if (columnName != null) {
if (columnName.isEmpty()) {
columnName = new StringBuilder().append(quotingChar).append(quotingChar).toString();
columnName = String.valueOf(quotingChar) + quotingChar;
} else if (columnName.charAt(0) != quotingChar
&& columnName.charAt(columnName.length() - 1) != quotingChar) {
columnName = columnName.replace("" + quotingChar, "" + quotingChar + quotingChar);
Expand All @@ -250,8 +247,15 @@ protected String[] supportedTableTypes() {
return new String[]{"TABLE"};
}

@Override
protected List<String> readPrimaryKeyOrUniqueIndexNames(java.sql.DatabaseMetaData metadata,
TableId id)
throws SQLException {
return readPrimaryKeyNames(metadata, id);
}

public enum OBSERVE_OUTPUT_FORMAT {
SQL, JSON;
SQL, JSON
}

public static class SingleStoreConnectionConfiguration {
Expand All @@ -277,6 +281,7 @@ public SingleStoreConnectionConfiguration(Configuration config) {
.with("sslMode", sslMode().getValue())
.with("defaultFetchSize", 1)
.with("tinyInt1IsBit", "false")
.with("enableExtendedDataTypes", "true")
.with("connectionAttributes", String.format(
"_connector_name:%s,_connector_version:%s,_product_version:%s",
"SingleStore Debezium Connector", Module.version(), Module.debeziumVersion()))
Expand Down Expand Up @@ -387,11 +392,4 @@ public CommonConnectorConfig.EventProcessingFailureHandlingMode eventProcessingF
return CommonConnectorConfig.EventProcessingFailureHandlingMode.parse(mode);
}
}

@Override
protected List<String> readPrimaryKeyOrUniqueIndexNames(java.sql.DatabaseMetaData metadata,
TableId id)
throws SQLException {
return readPrimaryKeyNames(metadata, id);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,6 @@
import io.debezium.relational.DefaultValueConverter;
import io.debezium.relational.ValueConverter;
import io.debezium.util.Strings;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.time.LocalDate;
Expand All @@ -22,6 +15,12 @@
import java.util.Optional;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class SingleStoreDefaultValueConverter implements DefaultValueConverter {

Expand Down Expand Up @@ -114,6 +113,7 @@ public Object convert(Column column, String value) {
case "FLOAT UNSIGNED":
return Float.parseFloat(value);
case "BIT":
case "BSON":
return value.getBytes(StandardCharsets.UTF_8);
case "DOUBLE":
case "DOUBLE UNSIGNED":
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ public SchemaBuilder schemaBuilder(Column column) {
case "LONGBLOB":
case "MEDIUMBLOB":
case "BLOB":
case "BSON":
return SchemaBuilder.bytes();
}
SchemaBuilder builder = super.schemaBuilder(column);
Expand Down Expand Up @@ -150,6 +151,7 @@ public ValueConverter converter(Column column, Field fieldDefn) {
case "LONGBLOB":
case "MEDIUMBLOB":
case "BLOB":
case "BSON":
return data -> convertBlob(column, fieldDefn, data);
}
return super.converter(column, fieldDefn);
Expand Down
Original file line number Diff line number Diff line change
@@ -1,35 +1,45 @@
package com.singlestore.debezium;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;

import com.singlestore.debezium.SingleStoreValueConverters.GeographyMode;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.jdbc.JdbcValueConverters;
import io.debezium.jdbc.TemporalPrecisionMode;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.relational.Tables;
import org.apache.kafka.connect.data.Struct;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import org.locationtech.jts.io.ParseException;

import java.nio.ByteBuffer;
import java.sql.Date;
import java.sql.SQLException;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.util.Optional;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.*;
import org.apache.kafka.connect.data.Struct;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import org.locationtech.jts.io.ParseException;

public class SingleStoreDefaultValueConverterIT extends IntegrationTestBase {

private static final SingleStoreValueConverters CONVERTERS = new SingleStoreValueConverters(
JdbcValueConverters.DecimalMode.DOUBLE, TemporalPrecisionMode.ADAPTIVE,
CommonConnectorConfig.BinaryHandlingMode.BYTES, GeographyMode.GEOMETRY);

private static void testColumn(SingleStoreDefaultValueConverter defaultValueConverter,
Table table, String name, Object expectedValue) {
Column column = table.columnWithName(name);
Optional<Object> defaultValue = defaultValueConverter.parseDefaultValue(column,
column.defaultValueExpression().orElse(null));
assertTrue(defaultValue.isPresent());
assertEquals(expectedValue, defaultValue.get());
}

@Test
public void testNumberValues() {
try (SingleStoreConnection conn = new SingleStoreConnection(defaultJdbcConnectionConfig())) {
Expand Down Expand Up @@ -192,17 +202,11 @@ public void testBlobValues() {
testColumn(defaultValueConverter, table, "mediumblobColumn",
ByteBuffer.wrap("abc".getBytes()));
testColumn(defaultValueConverter, table, "tinyblobColumn", ByteBuffer.wrap("abc".getBytes()));
// TODO: DB-78614 - default values for BSON columns omit trailing zeros
byte[] bsonColumnData = {5};
testColumn(defaultValueConverter, table, "bsonColumn", ByteBuffer.wrap(bsonColumnData));
} catch (SQLException e) {
Assert.fail(e.getMessage());
}
}

private static void testColumn(SingleStoreDefaultValueConverter defaultValueConverter,
Table table, String name, Object expectedValue) {
Column column = table.columnWithName(name);
Optional<Object> defaultValue = defaultValueConverter.parseDefaultValue(column,
column.defaultValueExpression().orElse(null));
assertTrue(defaultValue.isPresent());
assertEquals(expectedValue, defaultValue.get());
}
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,9 @@
package com.singlestore.debezium;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;

import com.singlestore.debezium.SingleStoreValueConverters.GeographyMode;
import com.singlestore.jdbc.SingleStoreBlob;
import io.debezium.config.CommonConnectorConfig;
Expand All @@ -9,30 +13,42 @@
import io.debezium.relational.Table;
import io.debezium.relational.Tables;
import io.debezium.util.HexConverter;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Struct;
import org.junit.Assert;
import org.junit.Test;
import org.locationtech.jts.io.ParseException;

import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.SQLException;
import java.time.*;
import java.time.Duration;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneId;
import java.util.Base64;
import java.util.Date;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Struct;
import org.junit.Assert;
import org.junit.Test;
import org.locationtech.jts.io.ParseException;

public class SingleStoreValueConvertersIT extends IntegrationTestBase {

private static final SingleStoreValueConverters CONVERTERS = new SingleStoreValueConverters(
JdbcValueConverters.DecimalMode.DOUBLE, TemporalPrecisionMode.CONNECT,
CommonConnectorConfig.BinaryHandlingMode.BYTES, GeographyMode.GEOMETRY);

private static void testColumn(SingleStoreValueConverters converters, Table table, String name,
Object valueToConvert, Object expectedConvertedValue) {
assertEquals(expectedConvertedValue,
convertColumnValue(converters, table, name, valueToConvert));
}

private static Object convertColumnValue(SingleStoreValueConverters converters, Table table,
String name, Object valueToConvert) {
Column column = table.columnWithName(name);
Field field = new Field(column.name(), -1, converters.schemaBuilder(column).build());
return converters.converter(column, field).convert(valueToConvert);
}

@Test
public void testNumberValues() {
try (SingleStoreConnection conn = new SingleStoreConnection(defaultJdbcConnectionConfig())) {
Expand Down Expand Up @@ -238,6 +254,21 @@ public void testBlobValues() {
}
}

@Test
public void testBsonValue() {
try (SingleStoreConnection conn = new SingleStoreConnection(defaultJdbcConnectionConfig())) {
Tables tables = new Tables();
conn.readSchema(tables, TEST_DATABASE, null, null, null, true);
Table table = tables.forTable(TEST_DATABASE, null, "allTypesTable");
assertThat(table).isNotNull();
byte[] bsonValue = {5, 0, 0, 0, 0};
SingleStoreBlob blob = new SingleStoreBlob(bsonValue);
testColumn(CONVERTERS, table, "bsonColumn", blob, ByteBuffer.wrap(bsonValue));
} catch (SQLException e) {
Assert.fail(e.getMessage());
}
}

@Test
public void testBinaryMode() {
testBinaryMode(CommonConnectorConfig.BinaryHandlingMode.BYTES);
Expand Down Expand Up @@ -281,17 +312,4 @@ private void testBinaryMode(CommonConnectorConfig.BinaryHandlingMode mode) {
Assert.fail(e.getMessage());
}
}

private static void testColumn(SingleStoreValueConverters converters, Table table, String name,
Object valueToConvert, Object expectedConvertedValue) {
assertEquals(expectedConvertedValue,
convertColumnValue(converters, table, name, valueToConvert));
}

private static Object convertColumnValue(SingleStoreValueConverters converters, Table table,
String name, Object valueToConvert) {
Column column = table.columnWithName(name);
Field field = new Field(column.name(), -1, converters.schemaBuilder(column).build());
return converters.converter(column, field).convert(valueToConvert);
}
}
Loading
Loading