diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index eee276d..41b2a6c 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -9,6 +9,7 @@ on: push: branches: - main + - test concurrency: group: ci-${{ github.event.pull_request.number || github.ref }} @@ -29,6 +30,10 @@ jobs: uses: ./.github/workflows/test.yml with: module: flink-connector-oceanbase-directload + flink-connector-oceanbase-tools-cdc: + uses: ./.github/workflows/test.yml + with: + module: flink-connector-oceanbase-tools-cdc flink-connector-oceanbase-e2e-tests: strategy: diff --git a/flink-connector-oceanbase-tools-cdc/pom.xml b/flink-connector-oceanbase-tools-cdc/pom.xml new file mode 100644 index 0000000..0692a87 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/pom.xml @@ -0,0 +1,123 @@ + + + + 4.0.0 + + + com.oceanbase + flink-connector-oceanbase-parent + ${revision} + + + flink-connector-oceanbase-tools-cdc + jar + + + + com.oceanbase + flink-connector-oceanbase + ${project.version} + + + com.oceanbase + flink-connector-oceanbase-base + ${project.version} + test-jar + test + + + + org.apache.flink + flink-connector-base + ${flink.version} + test + + + org.testcontainers + mysql + ${testcontainer.version} + test + + + org.apache.flink + flink-sql-connector-mysql-cdc + ${flink.sql.cdc.version} + provided + + + org.apache.flink + flink-shaded-guava + + + + + org.apache.flink + flink-sql-connector-oracle-cdc + ${flink.sql.cdc.version} + provided + + + org.apache.flink + flink-shaded-guava + + + + + org.apache.flink + flink-sql-connector-postgres-cdc + ${flink.sql.cdc.version} + provided + + + org.apache.flink + flink-shaded-guava + + + + + + org.apache.flink + flink-sql-connector-sqlserver-cdc + ${flink.sql.cdc.version} + provided + + + org.apache.flink + flink-shaded-guava + + + + + org.apache.flink + flink-sql-connector-db2-cdc + ${flink.sql.cdc.version} + provided + + + org.apache.flink + flink-shaded-guava + + + + + com.oracle.ojdbc + ojdbc8 + ${ojdbc.version} + provided + + + + diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/connection/OceanBaseToolsConnectProvider.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/connection/OceanBaseToolsConnectProvider.java new file mode 100644 index 0000000..444602f --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/connection/OceanBaseToolsConnectProvider.java @@ -0,0 +1,44 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.connection; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; +import com.oceanbase.connector.flink.utils.OceanBaseToolsJdbcUtils; + +public class OceanBaseToolsConnectProvider extends OceanBaseConnectionProvider { + + public OceanBaseToolsConnectProvider(OceanBaseConnectorOptions options) { + super(options); + } + + public boolean databaseExists(String database) { + return OceanBaseToolsJdbcUtils.databaseExists(database, this::getConnection); + } + + public void createDatabase(String database) { + OceanBaseToolsJdbcUtils.createDatabase(database, this::getConnection); + } + + public boolean tableExists(String database, String table) { + return OceanBaseToolsJdbcUtils.tableExists(database, table, this::getConnection); + } + + public void createTable(TableSchema schema) { + OceanBaseToolsJdbcUtils.createTable(schema, this::getConnection); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/table/OceanBaseJsonSerializationSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/table/OceanBaseJsonSerializationSchema.java new file mode 100644 index 0000000..75c8f7a --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/table/OceanBaseJsonSerializationSchema.java @@ -0,0 +1,174 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.table; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Time; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneId; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class OceanBaseJsonSerializationSchema extends AbstractRecordSerializationSchema { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = + LoggerFactory.getLogger(OceanBaseJsonSerializationSchema.class); + + private final TableInfo tableInfo; + private static final ObjectMapper objectMapper = new ObjectMapper(); + + public OceanBaseJsonSerializationSchema(TableInfo tableInfo) { + this.tableInfo = tableInfo; + } + + @Override + public Record serialize(String rowDataStr) { + try { + JsonNode rowDataNode = objectMapper.readTree(rowDataStr); + DataChangeRecord.Type type; + String op = rowDataNode.path("op").asText(); + if ("r".equals(op) || "c".equals(op)) { + type = DataChangeRecord.Type.UPSERT; + } else if ("d".equals(op)) { + type = DataChangeRecord.Type.DELETE; + } else { + throw new IllegalArgumentException("Unknown operation type: " + op); + } + int size = tableInfo.getFieldNames().size(); + Object[] values = new Object[size]; + for (int i = 0; i < size; i++) { + String fieldName = tableInfo.getFieldNames().get(i); + JsonNode fieldNode = rowDataNode.path("after").path(fieldName); + values[i] = objectMapper.convertValue(fieldNode, new TypeReference() {}); + } + + return new DataChangeRecord(tableInfo, type, values); + } catch (IOException e) { + String errorMessage = "Failed to parse rowData JSON: " + rowDataStr; + LOG.error(errorMessage, e); + throw new RuntimeException(errorMessage); + } + } + + @Override + protected SerializationRuntimeConverter createNotNullConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case BOOLEAN: + case TINYINT: + case SMALLINT: + case INTEGER: + case INTERVAL_YEAR_MONTH: + case BIGINT: + case INTERVAL_DAY_TIME: + case FLOAT: + case DOUBLE: + case BINARY: + case VARBINARY: + return data -> data; + case CHAR: + case VARCHAR: + return Object::toString; + case DATE: + return data -> Date.valueOf(LocalDate.ofEpochDay((int) data)); + case TIME_WITHOUT_TIME_ZONE: + return data -> Time.valueOf(LocalTime.ofNanoOfDay((int) data * 1_000_000L)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return data -> ((TimestampData) data).toTimestamp(); + case TIMESTAMP_WITH_TIME_ZONE: + return data -> ((TimestampData) data).toInstant().toString(); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return data -> + ((TimestampData) data) + .toInstant() + .atZone(ZoneId.systemDefault()) + .toString(); + case DECIMAL: + return data -> ((DecimalData) data).toBigDecimal(); + case ARRAY: + return data -> { + ArrayData arrayData = (ArrayData) data; + return IntStream.range(0, arrayData.size()) + .mapToObj(i -> arrayData.getString(i).toString()) + .collect(Collectors.joining(",")); + }; + case MAP: + return data -> { + MapData mapData = (MapData) data; + ArrayData keyArray = mapData.keyArray(); + ArrayData valueArray = mapData.valueArray(); + return "{" + + IntStream.range(0, keyArray.size()) + .mapToObj( + i -> + "\"" + + keyArray.getString(i).toString() + + "\":\"" + + valueArray.getString(i).toString() + + "\"") + .collect(Collectors.joining(",")) + + "}"; + }; + case ROW: + return data -> { + RowData rowData = (RowData) data; + RowType rowType = (RowType) type; + StringBuilder sb = new StringBuilder(); + sb.append("{"); + for (int i = 0; i < rowData.getArity(); i++) { + if (i > 0) { + sb.append(","); + } + String fieldName = rowType.getFieldNames().get(i); + LogicalType fieldType = rowType.getFields().get(i).getType(); + sb.append("\"").append(fieldName).append("\":"); + if (fieldType instanceof VarCharType) { + sb.append("\"").append(rowData.getString(i).toString()).append("\""); + } else if (fieldType instanceof IntType) { + sb.append(rowData.getInt(i)); + } else if (fieldType instanceof BooleanType) { + sb.append(rowData.getBoolean(i)); + } + // Add more types as needed + } + sb.append("}"); + return sb.toString(); + }; + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/FieldSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/FieldSchema.java new file mode 100644 index 0000000..4bd1e67 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/FieldSchema.java @@ -0,0 +1,75 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +public class FieldSchema { + private final String name; + private final String typeString; + private final String defaultValue; + private final String comment; + private final Boolean nullable; + + public FieldSchema( + String name, String typeString, String defaultValue, String comment, Boolean nullable) { + this.name = name; + this.typeString = typeString; + this.defaultValue = defaultValue; + this.comment = comment; + this.nullable = nullable; + } + + public String getName() { + return name; + } + + public String getTypeString() { + return typeString; + } + + public String getComment() { + return comment; + } + + public String getDefaultValue() { + return defaultValue; + } + + public Boolean getNullable() { + return nullable; + } + + @Override + public String toString() { + return "FieldSchema{" + + "name='" + + name + + '\'' + + ", typeString='" + + typeString + + '\'' + + ", defaultValue='" + + defaultValue + + '\'' + + ", comment='" + + comment + + '\'' + + ", nullable='" + + nullable + + '\'' + + '}'; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseSchemaFactory.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseSchemaFactory.java new file mode 100644 index 0000000..2a4f9a9 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseSchemaFactory.java @@ -0,0 +1,164 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.catalog; // Licensed to the Apache Software Foundation + +import com.oceanbase.connector.flink.dialect.OceanBaseDialect; +import com.oceanbase.connector.flink.dialect.OceanBaseMySQLDialect; + +import org.apache.flink.util.Preconditions; + +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; + +/** + * Factory that creates oceanbase schema. + * + * In the case where oceanbase schema needs to be created, it is best to create it through this + * factory + */ +public class OceanBaseSchemaFactory { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseSchemaFactory.class); + private static final OceanBaseDialect dialect = new OceanBaseMySQLDialect(); + + public static TableSchema createTableSchema( + String database, + String table, + Map columnFields, + List pkKeys, + String tableComment) { + TableSchema tableSchema = new TableSchema(); + tableSchema.setDatabase(database); + tableSchema.setTable(table); + tableSchema.setFields(columnFields); + tableSchema.setKeys(buildKeys(pkKeys, columnFields)); + tableSchema.setTableComment(tableComment); + return tableSchema; + } + + private static List buildDistributeKeys( + List primaryKeys, Map fields) { + return buildKeys(primaryKeys, fields); + } + + /** + * Theoretically, the duplicate table of oceanbase does not need to distinguish the key column, + * but in the actual table creation statement, the key column will be automatically added. So if + * it is a duplicate table, primaryKeys is empty, and we uniformly take the first field as the + * key. + */ + private static List buildKeys( + List primaryKeys, Map fields) { + if (CollectionUtils.isNotEmpty(primaryKeys)) { + return primaryKeys; + } + if (!fields.isEmpty()) { + Entry firstField = fields.entrySet().iterator().next(); + return Collections.singletonList(firstField.getKey()); + } + return new ArrayList<>(); + } + + public static String generateCreateTableDDL(TableSchema schema) { + StringBuilder sb = new StringBuilder("CREATE TABLE IF NOT EXISTS "); + sb.append(dialect.quoteIdentifier(schema.getDatabase())) + .append(".") + .append(dialect.quoteIdentifier(schema.getTable())) + .append(" ("); + + Map fields = schema.getFields(); + List keys = schema.getKeys(); + + // Append fields + for (Map.Entry entry : fields.entrySet()) { + FieldSchema field = entry.getValue(); + buildColumn(sb, field, keys.contains(entry.getKey())); + } + + sb = sb.deleteCharAt(sb.length() - 1); + // Append primary key constraint + if (!keys.isEmpty()) { + sb.append("PRIMARY KEY (") + .append( + keys.stream() + .map(dialect::quoteIdentifier) + .collect(Collectors.joining(","))) + .append(")"); + } + sb.append(")"); + + // Append table comment + if (schema.getTableComment() != null && !schema.getTableComment().trim().isEmpty()) { + sb.append(" COMMENT='").append(quoteComment(schema.getTableComment())).append("'"); + } + sb.append(";"); + + LOG.info("Generated DDL: {}", sb); + return sb.toString(); + } + + private static void buildColumn(StringBuilder sb, FieldSchema field, boolean isKey) { + sb.append(dialect.quoteIdentifier(field.getName())) + .append(" ") + .append(field.getTypeString()); + + if (!isKey && field.getNullable()) { + sb.append(" NULL"); + } else { + sb.append(" NOT NULL"); + } + + if (field.getDefaultValue() != null) { + sb.append(" DEFAULT ").append(quoteDefaultValue(field.getDefaultValue())); + } + + if (field.getComment() != null && !field.getComment().trim().isEmpty()) { + sb.append(" COMMENT '").append(quoteComment(field.getComment())).append("'"); + } + + sb.append(", "); + } + + public static String quoteDefaultValue(String defaultValue) { + // DEFAULT current_timestamp not need quote + if (defaultValue.equalsIgnoreCase("current_timestamp")) { + return defaultValue; + } + return "'" + defaultValue + "'"; + } + + public static String quoteComment(String comment) { + if (comment == null) { + return ""; + } else { + return comment.replaceAll("'", "\\\\'"); + } + } + + public static String quoteTableIdentifier(String tableIdentifier) { + String[] dbTable = tableIdentifier.split("\\."); + Preconditions.checkArgument(dbTable.length == 2); + return dialect.quoteIdentifier(dbTable[0]) + "." + dialect.quoteIdentifier(dbTable[1]); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java new file mode 100644 index 0000000..0827c6e --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java @@ -0,0 +1,41 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +public class OceanBaseType { + public static final String BOOLEAN = "BOOLEAN"; + public static final String TINYINT = "TINYINT"; + public static final String SMALLINT = "SMALLINT"; + public static final String INT = "INT"; + public static final String BIGINT = "BIGINT"; + public static final String LARGEINT = "LARGEINT"; + public static final String FLOAT = "FLOAT"; + public static final String DOUBLE = "DOUBLE"; + public static final String TIMESTAMP = "TIMESTAMP"; + public static final String DECIMAL = "DECIMAL"; + public static final String DATE = "DATE"; + public static final String DATETIME = "DATETIME"; + public static final String CHAR = "CHAR"; + public static final String VARCHAR = "VARCHAR"; + public static final String TEXT = "TEXT"; + public static final String ARRAY = "ARRAY"; + public static final String JSONB = "JSONB"; + public static final String JSON = "JSON"; + public static final String MAP = "MAP"; + public static final String STRUCT = "STRUCT"; + public static final String VARIANT = "VARIANT"; +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java new file mode 100644 index 0000000..83c08e7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java @@ -0,0 +1,187 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; + +import java.sql.Types; + +import static com.alibaba.druid.sql.ast.SQLDataType.Constants.BOOLEAN; +import static com.oceanbase.connector.flink.tools.catalog.OceanBaseType.*; + +public class OceanBaseTypeMapper { + + /** Max size of char type of OceanBase. */ + public static final int MAX_CHAR_SIZE = 256; + + /** Max size of varchar type of OceanBase. */ + public static final int MAX_VARCHAR_SIZE = 262144; + /* Max precision of datetime type of OceanBase. */ + public static final int MAX_SUPPORTED_DATE_TIME_PRECISION = 6; + + public static LogicalType convertToLogicalType(int jdbcType, int precision, int scale) { + switch (jdbcType) { + case Types.INTEGER: + return DataTypes.INT().getLogicalType(); + case Types.BIGINT: + return DataTypes.BIGINT().getLogicalType(); + case Types.DOUBLE: + return DataTypes.DOUBLE().getLogicalType(); + case Types.FLOAT: + case Types.REAL: + return DataTypes.FLOAT().getLogicalType(); + case Types.LONGVARCHAR: + case Types.VARCHAR: + if (precision > 0) { + return DataTypes.STRING().getLogicalType(); + } else { + return DataTypes.STRING().getLogicalType(); + } + case Types.CHAR: + if (precision > 0) { + return DataTypes.CHAR(precision).getLogicalType(); + } else { + return DataTypes.STRING().getLogicalType(); + } + case Types.TIMESTAMP: + if (precision > 0 && precision <= 3) { + return DataTypes.TIMESTAMP(precision).getLogicalType(); + } else { + return DataTypes.TIMESTAMP(3).getLogicalType(); + } + case Types.DATE: + return DataTypes.DATE().getLogicalType(); + case Types.TIME: + if (precision > 0 && precision <= 3) { + return DataTypes.TIME(precision).getLogicalType(); + } else { + return DataTypes.TIME(0).getLogicalType(); + } + case Types.DECIMAL: + case Types.NUMERIC: + if (precision > 0 && precision <= 38 && scale >= 0) { + return DataTypes.DECIMAL(precision, scale).getLogicalType(); + } else { + return DataTypes.DECIMAL(10, 0).getLogicalType(); + } + case Types.BOOLEAN: + case Types.BIT: + return DataTypes.BOOLEAN().getLogicalType(); + case Types.TINYINT: + return DataTypes.TINYINT().getLogicalType(); + case Types.SMALLINT: + return DataTypes.SMALLINT().getLogicalType(); + case Types.BLOB: + return DataTypes.BYTES().getLogicalType(); + case Types.CLOB: + return DataTypes.STRING().getLogicalType(); + case Types.BINARY: + if (precision > 0) { + return DataTypes.BINARY(precision).getLogicalType(); + } else { + return DataTypes.BYTES().getLogicalType(); + } + case Types.LONGVARBINARY: + case Types.VARBINARY: + if (precision > 0) { + return DataTypes.VARBINARY(precision).getLogicalType(); + } else { + return DataTypes.BYTES().getLogicalType(); + } + default: + throw new IllegalArgumentException("Unsupported JDBC type: " + jdbcType); + } + } + + public static String toOceanBaseType(DataType flinkType) { + LogicalType logicalType = flinkType.getLogicalType(); + return logicalType.accept(new LogicalTypeVisitor(logicalType)); + } + + private static class LogicalTypeVisitor extends LogicalTypeDefaultVisitor { + private final LogicalType type; + + LogicalTypeVisitor(LogicalType type) { + this.type = type; + } + + @Override + public String visit(CharType charType) { + long length = charType.getLength() * 3L; + if (length <= MAX_CHAR_SIZE) { + return String.format("%s(%s)", OceanBaseType.CHAR, length); + } else { + return visit(new VarCharType(charType.getLength())); + } + } + + @Override + public String visit(BooleanType booleanType) { + return BOOLEAN; + } + + @Override + public String visit(TinyIntType tinyIntType) { + return TINYINT; + } + + @Override + public String visit(SmallIntType smallIntType) { + return SMALLINT; + } + + @Override + public String visit(IntType intType) { + return INT; + } + + @Override + public String visit(BigIntType bigIntType) { + return BIGINT; + } + + @Override + public String visit(FloatType floatType) { + return FLOAT; + } + + @Override + public String visit(DoubleType doubleType) { + return DOUBLE; + } + + @Override + protected String defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support converting type %s to OceanBase type yet.", + type.toString())); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java new file mode 100644 index 0000000..818898a --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java @@ -0,0 +1,100 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TableSchema { + private String database; + private String table; + private String tableComment; + private Map fields; + private List keys = new ArrayList<>(); + private Map properties = new HashMap<>(); + + public String getDatabase() { + return database; + } + + public String getTable() { + return table; + } + + public String getTableComment() { + return tableComment; + } + + public Map getFields() { + return fields; + } + + public List getKeys() { + return keys; + } + + public Map getProperties() { + return properties; + } + + public void setDatabase(String database) { + this.database = database; + } + + public void setTable(String table) { + this.table = table; + } + + public void setTableComment(String tableComment) { + this.tableComment = tableComment; + } + + public void setFields(Map fields) { + this.fields = fields; + } + + public void setKeys(List keys) { + this.keys = keys; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + @Override + public String toString() { + return "TableSchema{" + + "database='" + + database + + '\'' + + ", table='" + + table + + '\'' + + ", tableComment='" + + tableComment + + '\'' + + ", fields=" + + fields + + ", keys=" + + String.join(",", keys) + + ", properties=" + + properties + + '}'; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java new file mode 100644 index 0000000..073cc08 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java @@ -0,0 +1,124 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; +import com.oceanbase.connector.flink.dialect.OceanBaseMySQLDialect; +import com.oceanbase.connector.flink.sink.OceanBaseRecordFlusher; +import com.oceanbase.connector.flink.sink.OceanBaseSink; +import com.oceanbase.connector.flink.table.DataChangeRecord; +import com.oceanbase.connector.flink.table.OceanBaseJsonSerializationSchema; +import com.oceanbase.connector.flink.table.TableId; +import com.oceanbase.connector.flink.table.TableInfo; + +import org.apache.flink.annotation.Public; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** OceanBase Sink Operate. */ +@Public +public class oceanBaseSinkOperate implements Serializable { + private static final long serialVersionUID = 1L; + protected Configuration sinkConfig; + private final OceanBaseConnectorOptions connectorOptions; + + public oceanBaseSinkOperate(Configuration cdcSinkConfig) { + sinkConfig = cdcSinkConfig; + this.connectorOptions = getOceanBaseConnectorOptions(); + } + + @Deprecated + public static String quoteDefaultValue(String defaultValue) { + return OceanBaseSchemaFactory.quoteDefaultValue(defaultValue); + } + + @Deprecated + public static String quoteComment(String comment) { + return OceanBaseSchemaFactory.quoteComment(comment); + } + + @Deprecated + public static String quoteTableIdentifier(String tableIdentifier) { + return OceanBaseSchemaFactory.quoteTableIdentifier(tableIdentifier); + } + + public OceanBaseSink createGenericOceanBaseSink(String schemaName, String tableName) + throws Exception { + + List columnNames = new ArrayList<>(); + List pkColumnNames = new ArrayList<>(); + List columnTypes = new ArrayList<>(); + OceanBaseMySQLDialect dialect = new OceanBaseMySQLDialect(); + try (OceanBaseConnectionProvider connectionProvider = + new OceanBaseConnectionProvider(connectorOptions)) { + Connection connection = connectionProvider.getConnection(); + DatabaseMetaData metaData = connection.getMetaData(); + try (ResultSet columns = metaData.getColumns(schemaName, schemaName, tableName, null)) { + while (columns.next()) { + String columnName = columns.getString("COLUMN_NAME"); + int dataType = columns.getInt("DATA_TYPE"); + int precision = columns.getInt("COLUMN_SIZE"); + int scale = columns.getInt("DECIMAL_DIGITS"); + columnNames.add(columnName); + columnTypes.add( + OceanBaseTypeMapper.convertToLogicalType(dataType, precision, scale)); + } + } + + try (ResultSet primaryKeys = + metaData.getPrimaryKeys(schemaName, schemaName, tableName)) { + while (primaryKeys.next()) { + String pkColumnName = primaryKeys.getString("COLUMN_NAME"); + pkColumnNames.add(pkColumnName); + } + } + } + TableInfo tableInfo = + new TableInfo( + new TableId(dialect::getFullTableName, schemaName, tableName), + pkColumnNames, + columnNames, + columnTypes, + null); + + return new OceanBaseSink<>( + connectorOptions, + null, + new OceanBaseJsonSerializationSchema(tableInfo), + DataChangeRecord.KeyExtractor.simple(), + new OceanBaseRecordFlusher(connectorOptions)); + } + + public OceanBaseConnectorOptions getOceanBaseConnectorOptions() { + Map options = new HashMap<>(); + if (sinkConfig != null) { + options = sinkConfig.toMap(); + return new OceanBaseConnectorOptions(options); + } + throw new RuntimeException("sinkConfig is null"); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java new file mode 100644 index 0000000..e73054c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java @@ -0,0 +1,215 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.cdc.db2.Db2DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.mysql.MysqlDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.oracle.OracleDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.postgres.PostgresDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.sqlserver.SqlServerDatabaseSync; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.utils.MultipleParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** cdc sync tools. */ +public class CdcTools { + private static final List EMPTY_KEYS = + Collections.singletonList(DatabaseSyncConfig.PASSWORD); + private static StreamExecutionEnvironment flinkEnvironmentForTesting; + private static JobClient jobClient; + + public static void main(String[] args) throws Exception { + System.out.println("Input args: " + Arrays.asList(args) + ".\n"); + String operation = args[0].toLowerCase(); + String[] opArgs = Arrays.copyOfRange(args, 1, args.length); + MultipleParameterTool params = MultipleParameterTool.fromArgs(opArgs); + switch (operation) { + // mysql is synchronized as a data source + case DatabaseSyncConfig.MYSQL_SYNC_DATABASE: + createMySQLSyncDatabase(params); + break; + // oracle is synchronized as a data source + case DatabaseSyncConfig.ORACLE_SYNC_DATABASE: + createOracleSyncDatabase(params); + break; + // postgres is synchronized as a data source + case DatabaseSyncConfig.POSTGRES_SYNC_DATABASE: + createPostgresSyncDatabase(params); + break; + // sqlserver is synchronized as a data source + case DatabaseSyncConfig.SQLSERVER_SYNC_DATABASE: + createSqlServerSyncDatabase(params); + break; + // db2 is synchronized as a data source + case DatabaseSyncConfig.DB2_SYNC_DATABASE: + createDb2SyncDatabase(params); + break; + default: + System.out.println("Unknown operation " + operation); + System.exit(1); + } + } + + private static void createMySQLSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.MYSQL_CONF)); + Map mysqlMap = getConfigMap(params, DatabaseSyncConfig.MYSQL_CONF); + Configuration mysqlConfig = Configuration.fromMap(mysqlMap); + DatabaseSync databaseSync = new MysqlDatabaseSync(); + syncDatabase(params, databaseSync, mysqlConfig, SourceConnector.MYSQL); + } + + private static void createOracleSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.ORACLE_CONF)); + Map oracleMap = getConfigMap(params, DatabaseSyncConfig.ORACLE_CONF); + Configuration oracleConfig = Configuration.fromMap(oracleMap); + DatabaseSync databaseSync = new OracleDatabaseSync(); + syncDatabase(params, databaseSync, oracleConfig, SourceConnector.ORACLE); + } + + private static void createPostgresSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.POSTGRES_CONF)); + Map postgresMap = getConfigMap(params, DatabaseSyncConfig.POSTGRES_CONF); + Configuration postgresConfig = Configuration.fromMap(postgresMap); + DatabaseSync databaseSync = new PostgresDatabaseSync(); + syncDatabase(params, databaseSync, postgresConfig, SourceConnector.POSTGRES); + } + + private static void createSqlServerSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.SQLSERVER_CONF)); + Map postgresMap = getConfigMap(params, DatabaseSyncConfig.SQLSERVER_CONF); + Configuration postgresConfig = Configuration.fromMap(postgresMap); + DatabaseSync databaseSync = new SqlServerDatabaseSync(); + syncDatabase(params, databaseSync, postgresConfig, SourceConnector.SQLSERVER); + } + + private static void createDb2SyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.DB2_CONF)); + Map db2Map = getConfigMap(params, DatabaseSyncConfig.DB2_CONF); + Configuration db2Config = Configuration.fromMap(db2Map); + DatabaseSync databaseSync = new Db2DatabaseSync(); + syncDatabase(params, databaseSync, db2Config, SourceConnector.DB2); + } + + private static void syncDatabase( + MultipleParameterTool params, + DatabaseSync databaseSync, + Configuration config, + SourceConnector sourceConnector) + throws Exception { + String jobName = params.get(DatabaseSyncConfig.JOB_NAME); + String database = params.get(DatabaseSyncConfig.DATABASE); + String tablePrefix = params.get(DatabaseSyncConfig.TABLE_PREFIX); + String tableSuffix = params.get(DatabaseSyncConfig.TABLE_SUFFIX); + String includingTables = params.get(DatabaseSyncConfig.INCLUDING_TABLES); + String excludingTables = params.get(DatabaseSyncConfig.EXCLUDING_TABLES); + String multiToOneOrigin = params.get(DatabaseSyncConfig.MULTI_TO_ONE_ORIGIN); + String multiToOneTarget = params.get(DatabaseSyncConfig.MULTI_TO_ONE_TARGET); + boolean createTableOnly = params.has(DatabaseSyncConfig.CREATE_TABLE_ONLY); + boolean ignoreDefaultValue = params.has(DatabaseSyncConfig.IGNORE_DEFAULT_VALUE); + boolean ignoreIncompatible = params.has(DatabaseSyncConfig.IGNORE_INCOMPATIBLE); + + Preconditions.checkArgument(params.has(DatabaseSyncConfig.SINK_CONF)); + Map sinkMap = getConfigMap(params, DatabaseSyncConfig.SINK_CONF); + Configuration sinkConfig = Configuration.fromMap(sinkMap); + + StreamExecutionEnvironment env = + Objects.nonNull(flinkEnvironmentForTesting) + ? flinkEnvironmentForTesting + : StreamExecutionEnvironment.getExecutionEnvironment(); + databaseSync + .setEnv(env) + .setDatabase(database) + .setConfig(config) + .setTablePrefix(tablePrefix) + .setTableSuffix(tableSuffix) + .setIncludingTables(includingTables) + .setExcludingTables(excludingTables) + .setMultiToOneOrigin(multiToOneOrigin) + .setMultiToOneTarget(multiToOneTarget) + .setIgnoreDefaultValue(ignoreDefaultValue) + .setSinkConfig(sinkConfig) + .setCreateTableOnly(createTableOnly) + .create(); + databaseSync.build(); + if (StringUtils.isNullOrWhitespaceOnly(jobName)) { + jobName = + String.format( + "%s-OceanBase Sync Database: %s", + sourceConnector.getConnectorName(), + config.getString( + DatabaseSyncConfig.DATABASE_NAME, DatabaseSyncConfig.DB)); + } + if (Objects.nonNull(flinkEnvironmentForTesting)) { + jobClient = env.executeAsync(); + } else { + env.execute(jobName); + } + } + + @VisibleForTesting + public static JobClient getJobClient() { + return jobClient; + } + + // Only for testing, please do not use it in actual environment + @VisibleForTesting + public static void setStreamExecutionEnvironmentForTesting( + StreamExecutionEnvironment environment) { + flinkEnvironmentForTesting = environment; + } + + @VisibleForTesting + public static Map getConfigMap(MultipleParameterTool params, String key) { + if (!params.has(key)) { + System.out.println( + "Can not find key [" + + key + + "] from args: " + + params.toMap().toString() + + ".\n"); + return null; + } + + Map map = new HashMap<>(); + for (String param : params.getMultiParameter(key)) { + String[] kv = param.split("=", 2); + if (kv.length == 2) { + map.put(kv[0].trim(), kv[1].trim()); + continue; + } else if (kv.length == 1 && EMPTY_KEYS.contains(kv[0])) { + map.put(kv[0].trim(), ""); + continue; + } + + System.out.println("Invalid " + key + " " + param + ".\n"); + return null; + } + return map; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java new file mode 100644 index 0000000..fb6b7e9 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java @@ -0,0 +1,401 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc; // Licensed to the Apache Software Foundation (ASF) + +import com.oceanbase.connector.flink.connection.OceanBaseToolsConnectProvider; +import com.oceanbase.connector.flink.tools.catalog.OceanBaseSchemaFactory; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; +import com.oceanbase.connector.flink.tools.catalog.oceanBaseSinkOperate; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.SQLSyntaxErrorException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.flink.cdc.debezium.utils.JdbcUrlUtils.PROPERTIES_PREFIX; + +public abstract class DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(DatabaseSync.class); + private static final String TABLE_NAME_OPTIONS = "table-name"; + + protected Configuration config; + + protected String database; + + protected TableNameConverter converter; + protected Pattern includingPattern; + protected Pattern excludingPattern; + protected Map multiToOneRulesPattern; + protected Configuration sinkConfig; + protected boolean ignoreDefaultValue; + protected boolean ignoreIncompatible; + + public StreamExecutionEnvironment env; + private boolean createTableOnly = false; + protected String includingTables; + protected String excludingTables; + protected String multiToOneOrigin; + protected String multiToOneTarget; + protected String tablePrefix; + protected String tableSuffix; + protected final Map tableMapping = new HashMap<>(); + public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; + + public abstract void registerDriver() throws SQLException; + + public abstract Connection getConnection() throws SQLException; + + public abstract List getSchemaList() throws Exception; + + public abstract DataStreamSource buildCdcSource(StreamExecutionEnvironment env); + + /** Get the prefix of a specific tableList, for example, mysql is database, oracle is schema. */ + public abstract String getTableListPrefix(); + + protected DatabaseSync() throws SQLException { + registerDriver(); + } + + public void create() { + this.includingPattern = includingTables == null ? null : Pattern.compile(includingTables); + this.excludingPattern = excludingTables == null ? null : Pattern.compile(excludingTables); + this.multiToOneRulesPattern = multiToOneRulesParser(multiToOneOrigin, multiToOneTarget); + this.converter = new TableNameConverter(tablePrefix, tableSuffix, multiToOneRulesPattern); + } + + public void build() throws Exception { + oceanBaseSinkOperate oceanBaseSinkOperate = new oceanBaseSinkOperate(sinkConfig); + OceanBaseToolsConnectProvider oceanBaseConnectionProvider = + new OceanBaseToolsConnectProvider( + oceanBaseSinkOperate.getOceanBaseConnectorOptions()); + List schemaList = getSchemaList(); + Preconditions.checkState( + !schemaList.isEmpty(), + "No tables to be synchronized. Please make sure whether the tables that need to be synchronized exist in the corresponding database or schema."); + + if (!StringUtils.isNullOrWhitespaceOnly(database) + && !oceanBaseConnectionProvider.databaseExists(database)) { + LOG.info("database {} not exist, created", database); + oceanBaseConnectionProvider.createDatabase(database); + } + List syncTables = new ArrayList<>(); + List> oceanbaseTables = new ArrayList<>(); + + Set targetDbSet = new HashSet<>(); + for (SourceSchema schema : schemaList) { + syncTables.add(schema.getTableName()); + String targetDb = database; + // Synchronize multiple databases using the src database name + if (StringUtils.isNullOrWhitespaceOnly(targetDb)) { + targetDb = schema.getDatabaseName(); + targetDbSet.add(targetDb); + } + if (StringUtils.isNullOrWhitespaceOnly(database) + && !oceanBaseConnectionProvider.databaseExists(targetDb)) { + LOG.info("database {} not exist, created", targetDb); + oceanBaseConnectionProvider.createDatabase(targetDb); + } + String oceanbaseTable = converter.convert(schema.getTableName()); + // Calculate the mapping relationship between upstream and downstream tables + tableMapping.put( + schema.getTableIdentifier(), String.format("%s.%s", targetDb, oceanbaseTable)); + tryCreateTableIfAbsent(oceanBaseConnectionProvider, targetDb, oceanbaseTable, schema); + + if (!oceanbaseTables.contains(Tuple2.of(targetDb, oceanbaseTable))) { + oceanbaseTables.add(Tuple2.of(targetDb, oceanbaseTable)); + } + } + if (createTableOnly) { + LOG.info("Create table finished."); + System.exit(0); + } + LOG.info("table mapping: {}", tableMapping); + config.setString(TABLE_NAME_OPTIONS, getSyncTableList(syncTables)); + DataStreamSource streamSource = buildCdcSource(env); + SingleOutputStreamOperator parsedStream = + streamSource.process(buildProcessFunction()); + for (Tuple2 dbTbl : oceanbaseTables) { + OutputTag recordOutputTag = + ParsingProcessFunction.createRecordOutputTag(dbTbl.f1); + DataStream sideOutput = parsedStream.getSideOutput(recordOutputTag); + int sinkParallel = sinkConfig.getInteger(SINK_PARALLELISM, sideOutput.getParallelism()); + String uidName = getUidName(targetDbSet, dbTbl); + sideOutput + .sinkTo(oceanBaseSinkOperate.createGenericOceanBaseSink(dbTbl.f0, dbTbl.f1)) + .setParallelism(sinkParallel) + .name(uidName) + .uid(uidName); + } + } + + /** + * @param targetDbSet The set of target databases. + * @param dbTbl The database-table tuple. + * @return The UID of the DataStream. + */ + public String getUidName(Set targetDbSet, Tuple2 dbTbl) { + String uidName; + // Determine whether to proceed with multi-database synchronization. + // if yes, the UID is composed of `dbname_tablename`, otherwise it is composed of + // `tablename`. + if (targetDbSet.size() > 1) { + uidName = dbTbl.f0 + "_" + dbTbl.f1; + } else { + uidName = dbTbl.f1; + } + + return uidName; + } + + public ParsingProcessFunction buildProcessFunction() { + return new ParsingProcessFunction(converter); + } + + /** Filter table that need to be synchronized. */ + protected boolean isSyncNeeded(String tableName) { + boolean sync = true; + if (includingPattern != null) { + sync = includingPattern.matcher(tableName).matches(); + } + if (excludingPattern != null) { + sync = sync && !excludingPattern.matcher(tableName).matches(); + } + LOG.debug("table {} is synchronized? {}", tableName, sync); + return sync; + } + + protected String getSyncTableList(List syncTables) { + // includingTablePattern and ^excludingPattern + if (includingTables == null) { + includingTables = ".*"; + } + String includingPattern = + String.format("(%s)\\.(%s)", getTableListPrefix(), includingTables); + if (StringUtils.isNullOrWhitespaceOnly(excludingTables)) { + return includingPattern; + } else { + String excludingPattern = + String.format("?!(%s\\.(%s))$", getTableListPrefix(), excludingTables); + return String.format("(%s)(%s)", excludingPattern, includingPattern); + } + } + + /** Filter table that many tables merge to one. */ + protected HashMap multiToOneRulesParser( + String multiToOneOrigin, String multiToOneTarget) { + if (StringUtils.isNullOrWhitespaceOnly(multiToOneOrigin) + || StringUtils.isNullOrWhitespaceOnly(multiToOneTarget)) { + return null; + } + HashMap multiToOneRulesPattern = new HashMap<>(); + String[] origins = multiToOneOrigin.split("\\|"); + String[] targets = multiToOneTarget.split("\\|"); + if (origins.length != targets.length) { + LOG.error( + "param error : multi to one params length are not equal,please check your params."); + System.exit(1); + } + try { + for (int i = 0; i < origins.length; i++) { + multiToOneRulesPattern.put(Pattern.compile(origins[i]), targets[i]); + } + } catch (Exception e) { + LOG.error("param error : Your regular expression is incorrect,please check."); + System.exit(1); + } + return multiToOneRulesPattern; + } + + private void tryCreateTableIfAbsent( + OceanBaseToolsConnectProvider connectionProvider, + String targetDb, + String oceanbaseTable, + SourceSchema schema) { + if (!connectionProvider.tableExists(targetDb, oceanbaseTable)) { + TableSchema oceanbaseSchema = + OceanBaseSchemaFactory.createTableSchema( + database, + oceanbaseTable, + schema.getFields(), + schema.getPrimaryKeys(), + schema.getTableComment()); + try { + connectionProvider.createTable(oceanbaseSchema); + } catch (Exception ex) { + handleTableCreationFailure(ex); + } + } + } + + private void handleTableCreationFailure(Exception ex) { + if (ignoreIncompatible && ex.getCause() instanceof SQLSyntaxErrorException) { + LOG.warn( + "OceanBase schema and source table schema are not compatible. Error: {} ", + ex.getCause().toString()); + } else { + throw new RuntimeException("Failed to create table due to: ", ex); + } + } + + protected Properties getJdbcProperties() { + Properties jdbcProps = new Properties(); + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(PROPERTIES_PREFIX)) { + jdbcProps.put(key.substring(PROPERTIES_PREFIX.length()), value); + } + } + return jdbcProps; + } + + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + jdbcProperties.forEach( + (key, value) -> jdbcUrlBuilder.append("&").append(key).append("=").append(value)); + return jdbcUrlBuilder.toString(); + } + + public DatabaseSync setEnv(StreamExecutionEnvironment env) { + this.env = env; + return this; + } + + public DatabaseSync setConfig(Configuration config) { + this.config = config; + return this; + } + + public DatabaseSync setDatabase(String database) { + this.database = database; + return this; + } + + public DatabaseSync setIncludingTables(String includingTables) { + this.includingTables = includingTables; + return this; + } + + public DatabaseSync setExcludingTables(String excludingTables) { + this.excludingTables = excludingTables; + return this; + } + + public DatabaseSync setMultiToOneOrigin(String multiToOneOrigin) { + this.multiToOneOrigin = multiToOneOrigin; + return this; + } + + public DatabaseSync setMultiToOneTarget(String multiToOneTarget) { + this.multiToOneTarget = multiToOneTarget; + return this; + } + + public DatabaseSync setSinkConfig(Configuration sinkConfig) { + this.sinkConfig = sinkConfig; + return this; + } + + public DatabaseSync setIgnoreDefaultValue(boolean ignoreDefaultValue) { + this.ignoreDefaultValue = ignoreDefaultValue; + return this; + } + + public DatabaseSync setCreateTableOnly(boolean createTableOnly) { + this.createTableOnly = createTableOnly; + return this; + } + + public DatabaseSync setTablePrefix(String tablePrefix) { + this.tablePrefix = tablePrefix; + return this; + } + + public DatabaseSync setTableSuffix(String tableSuffix) { + this.tableSuffix = tableSuffix; + return this; + } + + public static class TableNameConverter implements Serializable { + private static final long serialVersionUID = 1L; + private final String prefix; + private final String suffix; + private Map multiToOneRulesPattern; + + TableNameConverter() { + this("", ""); + } + + TableNameConverter(String prefix, String suffix) { + this.prefix = prefix == null ? "" : prefix; + this.suffix = suffix == null ? "" : suffix; + } + + TableNameConverter( + String prefix, String suffix, Map multiToOneRulesPattern) { + this.prefix = prefix == null ? "" : prefix; + this.suffix = suffix == null ? "" : suffix; + this.multiToOneRulesPattern = multiToOneRulesPattern; + } + + public String convert(String tableName) { + if (multiToOneRulesPattern == null) { + return prefix + tableName + suffix; + } + + String target = null; + + for (Map.Entry patternStringEntry : + multiToOneRulesPattern.entrySet()) { + if (patternStringEntry.getKey().matcher(tableName).matches()) { + target = patternStringEntry.getValue(); + } + } + /** + * If multiToOneRulesPattern is not null and target is not assigned, then the + * synchronization task contains both multi to one and one to one , prefixes and + * suffixes are added to common one-to-one mapping tables + */ + if (target == null) { + return prefix + tableName + suffix; + } + return target; + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java new file mode 100644 index 0000000..ec6b72d --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java @@ -0,0 +1,91 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +public class DatabaseSyncConfig { + + public static final String MYSQL_SYNC_DATABASE = "mysql-sync-database"; + public static final String ORACLE_SYNC_DATABASE = "oracle-sync-database"; + public static final String POSTGRES_SYNC_DATABASE = "postgres-sync-database"; + public static final String SQLSERVER_SYNC_DATABASE = "sqlserver-sync-database"; + public static final String DB2_SYNC_DATABASE = "db2-sync-database"; + + public static final String MYSQL_CONF = "mysql-conf"; + public static final String ORACLE_CONF = "oracle-conf"; + public static final String POSTGRES_CONF = "postgres-conf"; + public static final String SQLSERVER_CONF = "sqlserver-conf"; + public static final String DB2_CONF = "db2-conf"; + + ///////////// source-conf //////// + public static final String DATABASE_NAME = "database-name"; + public static final String DB = "db"; + public static final String USER = "user"; + public static final String PASSWORD = "password"; + public static final String TABLE_NAME = "TABLE_NAME"; + public static final String REMARKS = "REMARKS"; + + ////////// cdc-conf ////////// + // config options of {@link + // org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE} + public static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + public static final String SCAN_STARTUP_MODE_VALUE_EARLIEST_OFFSET = "earliest-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET = "specific-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET = "latest-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp"; + public static final String DECIMAL_HANDLING_MODE = "decimal.handling.mode"; + + ////////// sink-conf ///////////// + public static final String SINK_CONF = "sink-conf"; + public static final String JOB_NAME = "job-name"; + public static final String DATABASE = "database"; + public static final String TABLE_PREFIX = "table-prefix"; + public static final String TABLE_SUFFIX = "table-suffix"; + public static final String INCLUDING_TABLES = "including-tables"; + public static final String EXCLUDING_TABLES = "excluding-tables"; + public static final String MULTI_TO_ONE_ORIGIN = "multi-to-one-origin"; + public static final String MULTI_TO_ONE_TARGET = "multi-to-one-target"; + public static final String CREATE_TABLE_ONLY = "create-table-only"; + public static final String IGNORE_DEFAULT_VALUE = "ignore-default-value"; + public static final String IGNORE_INCOMPATIBLE = "ignore-incompatible"; + ////////// oceanbase-table-conf ////////// + public static final String TABLE_CONF = "table-conf"; + + ////////// date-converter-conf ////////// + public static final String CONVERTERS = "converters"; + public static final String DATE = "date"; + public static final String DATE_TYPE = "date.type"; + public static final String DATE_FORMAT_DATE = "date.format.date"; + public static final String DATE_FORMAT_DATETIME = "date.format.datetime"; + public static final String DATE_FORMAT_TIMESTAMP = "date.format.timestamp"; + public static final String DATE_FORMAT_TIMESTAMP_ZONE = "date.format.timestamp.zone"; + public static final String YEAR_MONTH_DAY_FORMAT = "yyyy-MM-dd"; + public static final String DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss"; + public static final String DATETIME_MICRO_FORMAT = "yyyy-MM-dd HH:mm:ss.SSSSSS"; + public static final String TIME_ZONE_SHANGHAI = "Asia/Shanghai"; + public static final String TIME_ZONE_UTC_8 = "UTC+8"; + public static final String FORMAT_DATE = "format.date"; + public static final String FORMAT_TIME = "format.time"; + public static final String FORMAT_DATETIME = "format.datetime"; + public static final String FORMAT_TIMESTAMP = "format.timestamp"; + public static final String FORMAT_TIMESTAMP_ZONE = "format.timestamp.zone"; + public static final String UPPERCASE_DATE = "DATE"; + public static final String TIME = "TIME"; + public static final String DATETIME = "DATETIME"; + public static final String TIMESTAMP = "TIMESTAMP"; + public static final String SMALLDATETIME = "SMALLDATETIME"; + public static final String DATETIME2 = "DATETIME2"; +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java new file mode 100644 index 0000000..4e26cff --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java @@ -0,0 +1,105 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; + +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; + +/** + * JdbcSourceSchema is a subclass of SourceSchema, used to build metadata about jdbc-related + * databases. + */ +public abstract class JdbcSourceSchema extends SourceSchema { + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceSchema.class); + + public JdbcSourceSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(databaseName, schemaName, tableName, tableComment); + fields = getColumnInfo(metaData, databaseName, schemaName, tableName); + primaryKeys = getPrimaryKeys(metaData, databaseName, schemaName, tableName); + } + + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + LinkedHashMap fields = new LinkedHashMap<>(); + LOG.debug("Starting to get column info for table: {}", tableName); + try (ResultSet rs = metaData.getColumns(databaseName, schemaName, tableName, null)) { + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + String comment = rs.getString("REMARKS"); + String fieldType = rs.getString("TYPE_NAME"); + Integer precision = rs.getInt("COLUMN_SIZE"); + Boolean isNullable = rs.getBoolean("IS_NULLABLE"); + String columnDefault = rs.getString("COLUMN_DEF"); // 默认值 + if (rs.wasNull()) { + precision = null; + } + Integer scale = rs.getInt("DECIMAL_DIGITS"); + if (rs.wasNull()) { + scale = null; + } + String oceanbaseTypeStr = null; + try { + oceanbaseTypeStr = convertToOceanBaseType(fieldType, precision, scale); + } catch (UnsupportedOperationException e) { + throw new UnsupportedOperationException(e + " in table: " + tableName); + } + fields.put( + fieldName, + new FieldSchema( + fieldName, oceanbaseTypeStr, columnDefault, comment, isNullable)); + } + } + Preconditions.checkArgument(!fields.isEmpty(), "The column info of {} is empty", tableName); + LOG.debug("Successfully retrieved column info for table: {}", tableName); + return fields; + } + + public List getPrimaryKeys( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + List primaryKeys = new ArrayList<>(); + try (ResultSet rs = metaData.getPrimaryKeys(databaseName, schemaName, tableName)) { + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } + + return primaryKeys; + } + + public abstract String convertToOceanBaseType( + String fieldType, Integer precision, Integer scale); +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java new file mode 100644 index 0000000..b64f041 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java @@ -0,0 +1,70 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.HashMap; +import java.util.Map; + +public class ParsingProcessFunction extends ProcessFunction { + protected ObjectMapper objectMapper = new ObjectMapper(); + private transient Map> recordOutputTags; + private DatabaseSync.TableNameConverter converter; + + public ParsingProcessFunction(DatabaseSync.TableNameConverter converter) { + this.converter = converter; + } + + @Override + public void open(Configuration parameters) throws Exception { + recordOutputTags = new HashMap<>(); + } + + @Override + public void processElement( + String record, ProcessFunction.Context context, Collector collector) + throws Exception { + String tableName = getRecordTableName(record); + String oceanbaseName = converter.convert(tableName); + context.output(getRecordOutputTag(oceanbaseName), record); + } + + protected String getRecordTableName(String record) throws Exception { + JsonNode recordRoot = objectMapper.readValue(record, JsonNode.class); + return extractJsonNode(recordRoot.get("source"), "table"); + } + + protected String extractJsonNode(JsonNode record, String key) { + return record != null && record.get(key) != null ? record.get(key).asText() : null; + } + + private OutputTag getRecordOutputTag(String tableName) { + return recordOutputTags.computeIfAbsent( + tableName, ParsingProcessFunction::createRecordOutputTag); + } + + public static OutputTag createRecordOutputTag(String tableName) { + return new OutputTag("record-" + tableName) {}; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java new file mode 100644 index 0000000..8d15b81 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java @@ -0,0 +1,36 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +public enum SourceConnector { + MYSQL("mysql"), + ORACLE("oracle"), + POSTGRES("postgres"), + SQLSERVER("sqlserver"), + MONGODB("mongodb"), + DB2("db2"); + + public final String connectorName; + + SourceConnector(String connectorName) { + this.connectorName = connectorName; + } + + public String getConnectorName() { + return connectorName; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java new file mode 100644 index 0000000..29debd2 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java @@ -0,0 +1,96 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; + +import org.apache.flink.util.StringUtils; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.StringJoiner; + +public abstract class SourceSchema { + protected final String databaseName; + protected final String schemaName; + protected final String tableName; + protected final String tableComment; + protected LinkedHashMap fields; + public List primaryKeys; + + public SourceSchema( + String databaseName, String schemaName, String tableName, String tableComment) + throws Exception { + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + this.tableComment = tableComment; + } + + public abstract String convertToOceanBaseType( + String fieldType, Integer precision, Integer scale); + + public String getTableIdentifier() { + return getString(databaseName, schemaName, tableName); + } + + public static String getString(String databaseName, String schemaName, String tableName) { + StringJoiner identifier = new StringJoiner("."); + if (!StringUtils.isNullOrWhitespaceOnly(databaseName)) { + identifier.add(databaseName); + } + if (!StringUtils.isNullOrWhitespaceOnly(schemaName)) { + identifier.add(schemaName); + } + + if (!StringUtils.isNullOrWhitespaceOnly(tableName)) { + identifier.add(tableName); + } + + return identifier.toString(); + } + + @Deprecated + public TableSchema convertTableSchema(Map tableProps) { + TableSchema tableSchema = new TableSchema(); + tableSchema.setFields(this.fields); + tableSchema.setTableComment(this.tableComment); + return tableSchema; + } + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public Map getFields() { + return fields; + } + + public List getPrimaryKeys() { + return primaryKeys; + } + + public String getTableComment() { + return tableComment; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java new file mode 100644 index 0000000..c7dc5bf --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java @@ -0,0 +1,241 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.db2.Db2Source; +import org.apache.flink.cdc.connectors.db2.source.Db2SourceBuilder; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SERVER_TIME_ZONE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class Db2DatabaseSync extends DatabaseSync { + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(50000) + .withDescription("Integer port number of the DB2 database server."); + private static final Logger LOG = LoggerFactory.getLogger(Db2DatabaseSync.class); + + private static final String JDBC_URL = "jdbc:db2://%s:%d/%s"; + + public Db2DatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.ibm.db2.jcc.DB2Driver"); + LOG.info(" Loaded the JDBC driver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class com.ibm.db2.jcc.DB2Driver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(JdbcSourceOptions.HOSTNAME), + config.get(PORT), + config.get(JdbcSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty("user", config.get(JdbcSourceOptions.USERNAME)); + pro.setProperty("password", config.get(JdbcSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(null, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString("TABLE_NAME"); + String tableComment = tables.getString("REMARKS"); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new Db2Schema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in DB2 is required"); + Preconditions.checkNotNull(schemaName, "schema-name in DB2 is required"); + + String tableName = config.get(JdbcSourceOptions.TABLE_NAME); + String hostname = config.get(JdbcSourceOptions.HOSTNAME); + Integer port = config.get(PORT); + String username = config.get(JdbcSourceOptions.USERNAME); + String password = config.get(JdbcSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(SourceOptions.SCAN_STARTUP_MODE); + if ("initial".equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if ("latest-offset".equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(Db2DateConverter.DEFAULT_PROPS); + debeziumProperties.put("decimal.handling.mode", "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, true)) { + JdbcIncrementalSource db2IncrementalSource = + Db2SourceBuilder.Db2IncrementalSource.builder() + .hostname(hostname) + .port(port) + .databaseList(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .deserializer(schema) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .serverTimeZone(config.get(SERVER_TIME_ZONE)) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + db2IncrementalSource, WatermarkStrategy.noWatermarks(), "Db2IncrementalSource"); + + } else { + DebeziumSourceFunction db2Source = + Db2Source.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(db2Source, "Db2 Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(JdbcSourceOptions.SCHEMA_NAME); + } + + @Override + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + boolean firstParam = true; + for (Map.Entry entry : jdbcProperties.entrySet()) { + Object key = entry.getKey(); + Object value = entry.getValue(); + if (firstParam) { + jdbcUrlBuilder.append(":").append(key).append("=").append(value).append(";"); + firstParam = false; + } else { + jdbcUrlBuilder.append(key).append("=").append(value).append(";"); + } + } + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java new file mode 100644 index 0000000..7f47bcd --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java @@ -0,0 +1,132 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class Db2DateConverter implements CustomConverter { + private static final Logger LOGGER = LoggerFactory.getLogger(Db2DateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + private final DateTimeFormatter timeFormatter = DateTimeFormatter.ISO_TIME; + + protected static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.db2.Db2DateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps(props, "format.date", p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + "format.timestamp", + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.isEmpty()) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + LOGGER.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } else if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof Time) { + return timeFormatter.format(((Time) input).toLocalTime()); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } else if (input instanceof Instant) { + LocalDateTime ldt = LocalDateTime.ofInstant(((Instant) input), ZoneOffset.UTC); + return timestampFormatter.format(ldt); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java new file mode 100644 index 0000000..3e416f3 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java @@ -0,0 +1,47 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; +import java.sql.SQLException; +import java.util.LinkedHashMap; + +public class Db2Schema extends JdbcSourceSchema { + public Db2Schema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return Db2Type.toOceanBaseType(fieldType, precision, scale); + } + + @Override + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + return super.getColumnInfo(metaData, null, schemaName, tableName); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java new file mode 100644 index 0000000..8e9fae4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java @@ -0,0 +1,93 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class Db2Type { + private static final String BOOLEAN = "BOOLEAN"; + private static final String SMALLINT = "SMALLINT"; + private static final String INTEGER = "INTEGER"; + private static final String INT = "INT"; + private static final String BIGINT = "BIGINT"; + private static final String REAL = "REAL"; + private static final String DECFLOAT = "DECFLOAT"; + private static final String DOUBLE = "DOUBLE"; + private static final String DECIMAL = "DECIMAL"; + private static final String NUMERIC = "NUMERIC"; + private static final String DATE = "DATE"; + private static final String TIME = "TIME"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String CHARACTER = "CHARACTER"; + private static final String CHAR = "CHAR"; + private static final String LONG_VARCHAR = "LONG VARCHAR"; + private static final String VARCHAR = "VARCHAR"; + private static final String XML = "XML"; + private static final String VARGRAPHIC = "VARGRAPHIC"; + + public static String toOceanBaseType(String db2Type, Integer precision, Integer scale) { + db2Type = db2Type.toUpperCase(); + switch (db2Type) { + case BOOLEAN: + return OceanBaseType.BOOLEAN; + case SMALLINT: + return OceanBaseType.SMALLINT; + case INTEGER: + case INT: + return OceanBaseType.INT; + case BIGINT: + return OceanBaseType.BIGINT; + case REAL: + return OceanBaseType.FLOAT; + case DOUBLE: + return OceanBaseType.DOUBLE; + case DATE: + return OceanBaseType.DATE; + case DECFLOAT: + case DECIMAL: + case NUMERIC: + if (precision != null && precision > 0 && precision <= 38) { + if (scale != null && scale >= 0) { + return String.format("%s(%s,%s)", OceanBaseType.DECIMAL, precision, scale); + } + return String.format("%s(%s,%s)", OceanBaseType.DECIMAL, precision, 0); + } else { + return OceanBaseType.VARCHAR; + } + case CHARACTER: + case CHAR: + case VARCHAR: + case LONG_VARCHAR: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case TIMESTAMP: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case TIME: + case VARGRAPHIC: + // Currently, the Flink CDC connector does not support the XML data type from DB2. + // Case XML: + return OceanBaseType.VARCHAR; + default: + throw new UnsupportedOperationException("Unsupported DB2 Type: " + db2Type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java new file mode 100644 index 0000000..42ca5b1 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java @@ -0,0 +1,196 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.deserialize; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.ConnectSchema; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Field; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Schema; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Struct; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.source.SourceRecord; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.util.Collector; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.Map; + +/** Currently just use for synchronous mysql non-default. */ +public class OceanBaseJsonDebeziumDeserializationSchema + implements DebeziumDeserializationSchema { + + private static final JsonNodeFactory JSON_NODE_FACTORY = + JsonNodeFactory.withExactBigDecimals(true); + private final ObjectMapper objectMapper; + + public OceanBaseJsonDebeziumDeserializationSchema() { + objectMapper = new ObjectMapper(); + } + + @Override + public void deserialize(SourceRecord sourceRecord, Collector collector) + throws Exception { + Schema schema = sourceRecord.valueSchema(); + Object value = sourceRecord.value(); + JsonNode jsonValue = convertToJson(schema, value); + byte[] bytes = objectMapper.writeValueAsString(jsonValue).getBytes(StandardCharsets.UTF_8); + collector.collect(new String(bytes)); + } + + private JsonNode convertToJson(Schema schema, Object value) throws RuntimeException { + if (value == null) { + // Any schema is valid and we don't have a default, so treat this as an optional schema + if (schema == null) { + return null; + } + if (schema.isOptional()) { + return JSON_NODE_FACTORY.nullNode(); + } + throw new RuntimeException( + "Conversion error: null value for field that is required and has no default value"); + } + + try { + final Schema.Type schemaType; + if (schema == null) { + schemaType = ConnectSchema.schemaType(value.getClass()); + if (schemaType == null) { + throw new RuntimeException( + "Java class " + + value.getClass() + + " does not have corresponding schema type."); + } + } else { + schemaType = schema.type(); + } + switch (schemaType) { + case INT8: + return JSON_NODE_FACTORY.numberNode((Byte) value); + case INT16: + return JSON_NODE_FACTORY.numberNode((Short) value); + case INT32: + return JSON_NODE_FACTORY.numberNode((Integer) value); + case INT64: + return JSON_NODE_FACTORY.numberNode((Long) value); + case FLOAT32: + return JSON_NODE_FACTORY.numberNode((Float) value); + case FLOAT64: + return JSON_NODE_FACTORY.numberNode((Double) value); + case BOOLEAN: + return JSON_NODE_FACTORY.booleanNode((Boolean) value); + case STRING: + CharSequence charSeq = (CharSequence) value; + return JSON_NODE_FACTORY.textNode(charSeq.toString()); + case BYTES: + if (value instanceof byte[]) { + return JSON_NODE_FACTORY.binaryNode((byte[]) value); + } else if (value instanceof ByteBuffer) { + return JSON_NODE_FACTORY.binaryNode(((ByteBuffer) value).array()); + } else if (value instanceof BigDecimal) { + return JSON_NODE_FACTORY.numberNode((BigDecimal) value); + } else { + throw new RuntimeException( + "Invalid type for bytes type: " + value.getClass()); + } + case ARRAY: + { + Collection> collection = (Collection>) value; + ArrayNode list = JSON_NODE_FACTORY.arrayNode(); + for (Object elem : collection) { + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode fieldValue = convertToJson(valueSchema, elem); + list.add(fieldValue); + } + return list; + } + case MAP: + { + Map, ?> map = (Map, ?>) value; + // If true, using string keys and JSON object; if false, using non-string + // keys and Array-encoding + boolean objectMode; + if (schema == null) { + objectMode = true; + for (Map.Entry, ?> entry : map.entrySet()) { + if (!(entry.getKey() instanceof String)) { + objectMode = false; + break; + } + } + } else { + objectMode = schema.keySchema().type() == Schema.Type.STRING; + } + ObjectNode obj = null; + ArrayNode list = null; + if (objectMode) { + obj = JSON_NODE_FACTORY.objectNode(); + } else { + list = JSON_NODE_FACTORY.arrayNode(); + } + for (Map.Entry, ?> entry : map.entrySet()) { + Schema keySchema = schema == null ? null : schema.keySchema(); + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode mapKey = convertToJson(keySchema, entry.getKey()); + JsonNode mapValue = convertToJson(valueSchema, entry.getValue()); + + if (objectMode) { + obj.set(mapKey.asText(), mapValue); + } else { + list.add(JSON_NODE_FACTORY.arrayNode().add(mapKey).add(mapValue)); + } + } + return objectMode ? obj : list; + } + case STRUCT: + { + Struct struct = (Struct) value; + if (!struct.schema().equals(schema)) { + throw new RuntimeException("Mismatching schema."); + } + ObjectNode obj = JSON_NODE_FACTORY.objectNode(); + for (Field field : schema.fields()) { + obj.set( + field.name(), + convertToJson( + field.schema(), + struct.getWithoutDefault(field.name()))); + } + return obj; + } + } + throw new RuntimeException("Couldn't convert " + value + " to JSON."); + } catch (ClassCastException e) { + String schemaTypeStr = (schema != null) ? schema.type().toString() : "unknown schema"; + throw new RuntimeException( + "Invalid type for " + schemaTypeStr + ": " + value.getClass()); + } + } + + @Override + public TypeInformation getProducedType() { + return BasicTypeInfo.STRING_TYPE_INFO; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java new file mode 100644 index 0000000..2f521f4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java @@ -0,0 +1,173 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class DateToStringConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(DateToStringConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timeFormatter = DateTimeFormatter.ISO_TIME; + private DateTimeFormatter datetimeFormatter = DateTimeFormatter.ISO_DATE_TIME; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + private ZoneId timestampZoneId = ZoneId.systemDefault(); + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.mysql.DateToStringConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATETIME, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP_ZONE, DatabaseSyncConfig.TIME_ZONE_UTC_8); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIME, + p -> timeFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_DATETIME, + p -> datetimeFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP_ZONE, + z -> timestampZoneId = ZoneId.of(z)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.DATETIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDateTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof Duration) { + Duration duration = (Duration) input; + long seconds = duration.getSeconds(); + int nano = duration.getNano(); + LocalTime time = LocalTime.ofSecondOfDay(seconds).withNano(nano); + return timeFormatter.format(time); + } + return null; + } + + private String convertDateTime(Object input) { + if (input instanceof LocalDateTime) { + return datetimeFormatter.format((LocalDateTime) input); + } else if (input instanceof Timestamp) { + return datetimeFormatter.format(((Timestamp) input).toLocalDateTime()); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof ZonedDateTime) { + // mysql timestamp will be converted to UTC storage, + // and the zonedDatetime here is UTC time + ZonedDateTime zonedDateTime = (ZonedDateTime) input; + LocalDateTime localDateTime = + zonedDateTime.withZoneSameInstant(timestampZoneId).toLocalDateTime(); + return timestampFormatter.format(localDateTime); + } else if (input instanceof Timestamp) { + return timestampFormatter.format( + ((Timestamp) input).toInstant().atZone(timestampZoneId).toLocalDateTime()); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java new file mode 100644 index 0000000..2b89ff4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java @@ -0,0 +1,264 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.mysql.source.MySqlSource; +import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceBuilder; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; +import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.apache.flink.cdc.debezium.utils.JdbcUrlUtils.PROPERTIES_PREFIX; + +public class MysqlDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(MysqlDatabaseSync.class); + private static final String JDBC_URL = "jdbc:mysql://%s:%d?useInformationSchema=true"; + + public MysqlDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.mysql.cj.jdbc.Driver"); + } catch (ClassNotFoundException ex) { + LOG.warn( + "can not found class com.mysql.cj.jdbc.Driver, use class com.mysql.jdbc.Driver"); + try { + Class.forName("com.mysql.jdbc.Driver"); + } catch (Exception e) { + throw new SQLException( + "No suitable driver found, can not found class com.mysql.cj.jdbc.Driver and com.mysql.jdbc.Driver"); + } + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(MySqlSourceOptions.HOSTNAME), + config.get(MySqlSourceOptions.PORT)); + return DriverManager.getConnection( + jdbcUrl, + config.get(MySqlSourceOptions.USERNAME), + config.get(MySqlSourceOptions.PASSWORD)); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(MySqlSourceOptions.DATABASE_NAME); + List schemaList = new ArrayList<>(); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet catalogs = metaData.getCatalogs()) { + while (catalogs.next()) { + String tableCatalog = catalogs.getString("TABLE_CAT"); + if (tableCatalog.matches(databaseName)) { + try (ResultSet tables = + metaData.getTables( + tableCatalog, null, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new MysqlSchema( + metaData, tableCatalog, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + MySqlSourceBuilder sourceBuilder = MySqlSource.builder(); + + String databaseName = config.get(MySqlSourceOptions.DATABASE_NAME); + Preconditions.checkNotNull(databaseName, "database-name in mysql is required"); + String tableName = config.get(MySqlSourceOptions.TABLE_NAME); + sourceBuilder + .hostname(config.get(MySqlSourceOptions.HOSTNAME)) + .port(config.get(MySqlSourceOptions.PORT)) + .username(config.get(MySqlSourceOptions.USERNAME)) + .password(config.get(MySqlSourceOptions.PASSWORD)) + .databaseList(databaseName) + .tableList(tableName); + + config.getOptional(MySqlSourceOptions.SERVER_ID).ifPresent(sourceBuilder::serverId); + config.getOptional(MySqlSourceOptions.SERVER_TIME_ZONE) + .ifPresent(sourceBuilder::serverTimeZone); + config.getOptional(MySqlSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE) + .ifPresent(sourceBuilder::fetchSize); + config.getOptional(MySqlSourceOptions.CONNECT_TIMEOUT) + .ifPresent(sourceBuilder::connectTimeout); + config.getOptional(MySqlSourceOptions.CONNECT_MAX_RETRIES) + .ifPresent(sourceBuilder::connectMaxRetries); + config.getOptional(MySqlSourceOptions.CONNECTION_POOL_SIZE) + .ifPresent(sourceBuilder::connectionPoolSize); + config.getOptional(MySqlSourceOptions.HEARTBEAT_INTERVAL) + .ifPresent(sourceBuilder::heartbeatInterval); + config.getOptional(MySqlSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED) + .ifPresent(sourceBuilder::scanNewlyAddedTableEnabled); + config.getOptional(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE) + .ifPresent(sourceBuilder::splitSize); + config.getOptional(MySqlSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED) + .ifPresent(sourceBuilder::closeIdleReaders); + + setChunkColumns(sourceBuilder); + String startupMode = config.get(MySqlSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + sourceBuilder.startupOptions(StartupOptions.initial()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_EARLIEST_OFFSET.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions(StartupOptions.earliest()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions(StartupOptions.latest()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET.equalsIgnoreCase( + startupMode)) { + BinlogOffsetBuilder offsetBuilder = BinlogOffset.builder(); + String file = config.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE); + Long pos = config.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_POS); + if (file != null && pos != null) { + offsetBuilder.setBinlogFilePosition(file, pos); + } + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_GTID_SET) + .ifPresent(offsetBuilder::setGtidSet); + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS) + .ifPresent(offsetBuilder::setSkipEvents); + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS) + .ifPresent(offsetBuilder::setSkipRows); + sourceBuilder.startupOptions(StartupOptions.specificOffset(offsetBuilder.build())); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_TIMESTAMP.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions( + StartupOptions.timestamp( + config.get(MySqlSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS))); + } + + Properties jdbcProperties = new Properties(); + Properties debeziumProperties = new Properties(); + // date to string + debeziumProperties.putAll(DateToStringConverter.DEFAULT_PROPS); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(PROPERTIES_PREFIX)) { + jdbcProperties.put(key.substring(PROPERTIES_PREFIX.length()), value); + } else if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + sourceBuilder.jdbcProperties(jdbcProperties); + sourceBuilder.debeziumProperties(debeziumProperties); + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + customConverterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric"); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + MySqlSource mySqlSource = + sourceBuilder.deserializer(schema).includeSchemaChanges(true).build(); + + return env.fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source"); + } + + @Override + public String getTableListPrefix() { + return config.get(MySqlSourceOptions.DATABASE_NAME); + } + + /** + * set chunkkeyColumn,eg: db.table1:column1,db.table2:column2. + * + * @param sourceBuilder + */ + private void setChunkColumns(MySqlSourceBuilder sourceBuilder) { + Map chunkColumnMap = getChunkColumnMap(); + for (Map.Entry entry : chunkColumnMap.entrySet()) { + sourceBuilder.chunkKeyColumn(entry.getKey(), entry.getValue()); + } + } + + private Map getChunkColumnMap() { + Map chunkMap = new HashMap<>(); + String chunkColumn = + config.getString(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN); + if (!StringUtils.isNullOrWhitespaceOnly(chunkColumn)) { + final Pattern chunkPattern = Pattern.compile("(\\S+)\\.(\\S+):(\\S+)"); + String[] tblColumns = chunkColumn.split(","); + for (String tblCol : tblColumns) { + Matcher matcher = chunkPattern.matcher(tblCol); + if (matcher.find()) { + String db = matcher.group(1); + String table = matcher.group(2); + String col = matcher.group(3); + chunkMap.put(new ObjectPath(db, table), col); + } + } + } + return chunkMap; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java new file mode 100644 index 0000000..0941b8c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java @@ -0,0 +1,34 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class MysqlSchema extends JdbcSourceSchema { + + public MysqlSchema( + DatabaseMetaData metaData, String databaseName, String tableName, String tableComment) + throws Exception { + super(metaData, databaseName, null, tableName, tableComment); + } + + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return MysqlType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java new file mode 100644 index 0000000..8f23ee7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java @@ -0,0 +1,176 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.mysql; + +public class MysqlType { + private static final String BIT = "BIT"; + private static final String BOOLEAN = "BOOLEAN"; + private static final String BOOL = "BOOL"; + private static final String TINYINT = "TINYINT"; + private static final String TINYINT_UNSIGNED = "TINYINT UNSIGNED"; + private static final String TINYINT_UNSIGNED_ZEROFILL = "TINYINT UNSIGNED ZEROFILL"; + private static final String SMALLINT = "SMALLINT"; + private static final String SMALLINT_UNSIGNED = "SMALLINT UNSIGNED"; + private static final String SMALLINT_UNSIGNED_ZEROFILL = "SMALLINT UNSIGNED ZEROFILL"; + private static final String MEDIUMINT = "MEDIUMINT"; + private static final String MEDIUMINT_UNSIGNED = "MEDIUMINT UNSIGNED"; + private static final String MEDIUMINT_UNSIGNED_ZEROFILL = "MEDIUMINT UNSIGNED ZEROFILL"; + private static final String INT = "INT"; + private static final String INT_UNSIGNED = "INT UNSIGNED"; + private static final String INT_UNSIGNED_ZEROFILL = "INT UNSIGNED ZEROFILL"; + private static final String INTEGER = "INTEGER"; + private static final String INTEGER_UNSIGNED = "INTEGER UNSIGNED"; + private static final String INTEGER_UNSIGNED_ZEROFILL = "INTEGER UNSIGNED ZEROFILL"; + private static final String BIGINT = "BIGINT"; + private static final String SERIAL = "SERIAL"; + private static final String BIGINT_UNSIGNED = "BIGINT UNSIGNED"; + private static final String BIGINT_UNSIGNED_ZEROFILL = "BIGINT UNSIGNED ZEROFILL"; + private static final String REAL = "REAL"; + private static final String REAL_UNSIGNED = "REAL UNSIGNED"; + private static final String REAL_UNSIGNED_ZEROFILL = "REAL UNSIGNED ZEROFILL"; + private static final String FLOAT = "FLOAT"; + private static final String FLOAT_UNSIGNED = "FLOAT UNSIGNED"; + private static final String FLOAT_UNSIGNED_ZEROFILL = "FLOAT UNSIGNED ZEROFILL"; + private static final String DOUBLE = "DOUBLE"; + private static final String DOUBLE_UNSIGNED = "DOUBLE UNSIGNED"; + private static final String DOUBLE_UNSIGNED_ZEROFILL = "DOUBLE UNSIGNED ZEROFILL"; + private static final String DOUBLE_PRECISION = "DOUBLE PRECISION"; + private static final String DOUBLE_PRECISION_UNSIGNED = "DOUBLE PRECISION UNSIGNED"; + private static final String DOUBLE_PRECISION_UNSIGNED_ZEROFILL = + "DOUBLE PRECISION UNSIGNED ZEROFILL"; + private static final String NUMERIC = "NUMERIC"; + private static final String NUMERIC_UNSIGNED = "NUMERIC UNSIGNED"; + private static final String NUMERIC_UNSIGNED_ZEROFILL = "NUMERIC UNSIGNED ZEROFILL"; + private static final String FIXED = "FIXED"; + private static final String FIXED_UNSIGNED = "FIXED UNSIGNED"; + private static final String FIXED_UNSIGNED_ZEROFILL = "FIXED UNSIGNED ZEROFILL"; + private static final String DECIMAL = "DECIMAL"; + private static final String DECIMAL_UNSIGNED = "DECIMAL UNSIGNED"; + private static final String DECIMAL_UNSIGNED_ZEROFILL = "DECIMAL UNSIGNED ZEROFILL"; + private static final String CHAR = "CHAR"; + private static final String VARCHAR = "VARCHAR"; + private static final String TINYTEXT = "TINYTEXT"; + private static final String MEDIUMTEXT = "MEDIUMTEXT"; + private static final String TEXT = "TEXT"; + private static final String LONGTEXT = "LONGTEXT"; + private static final String DATE = "DATE"; + private static final String TIME = "TIME"; + private static final String DATETIME = "DATETIME"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String YEAR = "YEAR"; + private static final String BINARY = "BINARY"; + private static final String VARBINARY = "VARBINARY"; + private static final String TINYBLOB = "TINYBLOB"; + private static final String MEDIUMBLOB = "MEDIUMBLOB"; + private static final String BLOB = "BLOB"; + private static final String LONGBLOB = "LONGBLOB"; + private static final String JSON = "JSON"; + private static final String ENUM = "ENUM"; + private static final String SET = "SET"; + + public static String toOceanBaseType(String type, Integer length, Integer scale) { + switch (type.toUpperCase()) { + case BIT: + return BIT; + case BOOLEAN: + case BOOL: + return BOOLEAN; + case TINYINT: + case TINYINT_UNSIGNED: + case TINYINT_UNSIGNED_ZEROFILL: + return TINYINT; + case SMALLINT: + case SMALLINT_UNSIGNED: + case SMALLINT_UNSIGNED_ZEROFILL: + return SMALLINT; + case MEDIUMINT: + case MEDIUMINT_UNSIGNED: + case MEDIUMINT_UNSIGNED_ZEROFILL: + return MEDIUMINT; + case INT: + case INTEGER: + case INT_UNSIGNED: + case INT_UNSIGNED_ZEROFILL: + case INTEGER_UNSIGNED: + case INTEGER_UNSIGNED_ZEROFILL: + return INT; + case BIGINT: + case BIGINT_UNSIGNED: + case BIGINT_UNSIGNED_ZEROFILL: + case SERIAL: + return BIGINT; + case REAL: + case REAL_UNSIGNED: + case REAL_UNSIGNED_ZEROFILL: + case FLOAT: + case FLOAT_UNSIGNED: + case FLOAT_UNSIGNED_ZEROFILL: + return FLOAT; + case DOUBLE: + case DOUBLE_UNSIGNED: + case DOUBLE_UNSIGNED_ZEROFILL: + case DOUBLE_PRECISION: + case DOUBLE_PRECISION_UNSIGNED: + case DOUBLE_PRECISION_UNSIGNED_ZEROFILL: + return DOUBLE; + case NUMERIC: + case NUMERIC_UNSIGNED: + case NUMERIC_UNSIGNED_ZEROFILL: + case FIXED: + case FIXED_UNSIGNED: + case FIXED_UNSIGNED_ZEROFILL: + case DECIMAL: + case DECIMAL_UNSIGNED: + case DECIMAL_UNSIGNED_ZEROFILL: + return DECIMAL + "(" + length + "," + scale + ")"; + case CHAR: + return CHAR + "(" + length + ")"; + case VARCHAR: + return VARCHAR + "(" + length + ")"; + case TINYTEXT: + case MEDIUMTEXT: + case TEXT: + case LONGTEXT: + return TEXT; + case DATE: + return DATE; + case TIME: + return TIME + "(" + scale + ")"; + case DATETIME: + return DATETIME + "(" + scale + ")"; + case TIMESTAMP: + return TIMESTAMP + "(" + scale + ")"; + case YEAR: + return YEAR; + case BINARY: + return BINARY + "(" + length + ")"; + case VARBINARY: + return VARBINARY + "(" + length + ")"; + case TINYBLOB: + case MEDIUMBLOB: + case BLOB: + case LONGBLOB: + return BLOB; + case JSON: + return JSON; + case ENUM: + case SET: + return type; // ENUM and SET types are kept as is + default: + throw new UnsupportedOperationException("Unsupported MySQL type: " + type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java new file mode 100644 index 0000000..fb050d6 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java @@ -0,0 +1,257 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.oracle.OracleSource; +import org.apache.flink.cdc.connectors.oracle.source.OracleSourceBuilder; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceOptions; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class OracleDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(OracleDatabaseSync.class); + + private static final String JDBC_URL = "jdbc:oracle:thin:@%s:%d:%s"; + private static final String PDB_KEY = "debezium.database.pdb.name"; + + public OracleDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("oracle.jdbc.driver.OracleDriver"); + } catch (ClassNotFoundException ex) { + LOG.warn( + "can not found class oracle.jdbc.driver.OracleDriver, use class oracle.jdbc.OracleDriver"); + try { + Class.forName("oracle.jdbc.OracleDriver"); + } catch (Exception e) { + throw new SQLException( + "No suitable driver found, can not found class oracle.jdbc.driver.OracleDriver and oracle.jdbc.OracleDriver"); + } + } + } + + @Override + public Connection getConnection() throws SQLException { + String jdbcUrl; + if (!StringUtils.isNullOrWhitespaceOnly(config.get(OracleSourceOptions.URL))) { + jdbcUrl = config.get(OracleSourceOptions.URL); + } else { + jdbcUrl = + String.format( + JDBC_URL, + config.get(OracleSourceOptions.HOSTNAME), + config.get(OracleSourceOptions.PORT), + config.get(OracleSourceOptions.DATABASE_NAME)); + } + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(OracleSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(OracleSourceOptions.PASSWORD)); + pro.put("remarksReporting", "true"); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(OracleSourceOptions.DATABASE_NAME); + String schemaName = config.get(OracleSourceOptions.SCHEMA_NAME); + + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + setSessionToPdb(conn); + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(databaseName, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new OracleSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + private void setSessionToPdb(Connection conn) throws SQLException { + String pdbName = null; + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + if (key.equals(PDB_KEY)) { + pdbName = entry.getValue(); + break; + } + } + if (!StringUtils.isNullOrWhitespaceOnly(pdbName)) { + LOG.info("Found pdb name in config, set session to pdb to {}", pdbName); + try (Statement statement = conn.createStatement()) { + statement.execute("alter session set container=" + pdbName); + } + } + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + Properties debeziumProperties = new Properties(); + String databaseName = config.get(OracleSourceOptions.DATABASE_NAME); + String schemaName = config.get(OracleSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in oracle is required"); + Preconditions.checkNotNull(schemaName, "schema-name in oracle is required"); + String tableName = config.get(OracleSourceOptions.TABLE_NAME); + // When debezium incrementally reads, it will be judged based on regexp_like. + // When the regular length exceeds 512, an error will be reported, + // like ORA-12733: regular expression too long + if (tableName.length() > 512) { + tableName = StringUtils.isNullOrWhitespaceOnly(includingTables) ? ".*" : tableName; + } + + String url = config.get(OracleSourceOptions.URL); + String hostname = config.get(OracleSourceOptions.HOSTNAME); + Integer port = config.get(OracleSourceOptions.PORT); + String username = config.get(OracleSourceOptions.USERNAME); + String password = config.get(OracleSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(OracleSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + // date to string + debeziumProperties.putAll(OracleDateConverter.DEFAULT_PROPS); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(OracleSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + OracleSourceBuilder.OracleIncrementalSource.builder() + .hostname(hostname) + .url(url) + .port(port) + .databaseList(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .includeSchemaChanges(true) + .startupOptions(startupOptions) + .deserializer(schema) + .debeziumProperties(debeziumProperties) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "Oracle IncrSource"); + } else { + DebeziumSourceFunction oracleSource = + OracleSource.builder() + .url(url) + .hostname(hostname) + .port(port) + .username(username) + .password(password) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(oracleSource, "Oracle Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(OracleSourceOptions.SCHEMA_NAME); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java new file mode 100644 index 0000000..c3776a2 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java @@ -0,0 +1,152 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import oracle.sql.TIMESTAMP; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class OracleDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(OracleDateConverter.class); + private static final Pattern TO_DATE = + Pattern.compile("TO_DATE\\('(.*)',[ ]*'(.*)'\\)", Pattern.CASE_INSENSITIVE); + private static final Pattern TO_TIMESTAMP = + Pattern.compile("TO_TIMESTAMP\\('(.*)'\\)", Pattern.CASE_INSENSITIVE); + private static final Pattern TIMESTAMP_OR_DATE_REGEX = + Pattern.compile("^TIMESTAMP[(]\\d[)]$|^DATE$", Pattern.CASE_INSENSITIVE); + private ZoneId timestampZoneId = ZoneId.systemDefault(); + public static final Properties DEFAULT_PROPS = new Properties(); + private static final String DATETIME_PATTERN = DatabaseSyncConfig.DATE_TIME_FORMAT; + private static final String DATETIMEV2_PATTERN = DatabaseSyncConfig.DATETIME_MICRO_FORMAT; + private final DateTimeFormatter dateTimeV2Formatter = + DateTimeFormatter.ofPattern(DATETIMEV2_PATTERN); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, "oracleDate"); + DEFAULT_PROPS.setProperty( + "oracleDate.type", + "com.oceanbase.connector.flink.tools.cdc.oracle.OracleDateConverter"); + } + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern(DatabaseSyncConfig.DATE_TIME_FORMAT) + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .toFormatter(); + + @Override + public void configure(Properties props) {} + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String typeName = column.typeName(); + if (TIMESTAMP_OR_DATE_REGEX.matcher(typeName).matches()) { + registration.register( + SchemaBuilder.string().optional(), + value -> { + if (value == null) { + if (column.isOptional()) { + return null; + } else if (column.hasDefaultValue()) { + return column.defaultValue(); + } else { + return null; + } + } + + if (value instanceof String) { + return convertStringTimestamp((String) value); + } + if (value instanceof Timestamp) { + return dateTimeV2Formatter.format( + ((Timestamp) value).toLocalDateTime()); + } + + // oracle timestamp + try { + if (value instanceof TIMESTAMP) { + return dateTimeV2Formatter.format( + ((TIMESTAMP) value).timestampValue().toLocalDateTime()); + } + } catch (SQLException ex) { + log.error("convert timestamp failed, values is {}", value); + } + + return null; + }); + } + } + + private String convertStringTimestamp(String data) { + LocalDateTime dateTime; + + final Matcher toTimestampMatcher = TO_TIMESTAMP.matcher(data); + if (toTimestampMatcher.matches()) { + String dateText = toTimestampMatcher.group(1); + dateTime = + LocalDateTime.from( + TIMESTAMP_FORMATTER.parse(completeMilliseconds(dateText.trim()))); + return dateTimeV2Formatter.format(dateTime.atZone(timestampZoneId)); + } + + final Matcher toDateMatcher = TO_DATE.matcher(data); + if (toDateMatcher.matches()) { + String date = toDateMatcher.group(1); + dateTime = + LocalDateTime.from( + TIMESTAMP_FORMATTER.parse(completeMilliseconds(date.trim()))); + return dateTimeV2Formatter.format(dateTime.atZone(timestampZoneId)); + } + return null; + } + + private String completeMilliseconds(String stringValue) { + if (stringValue.length() == DATETIMEV2_PATTERN.length()) { + return stringValue; + } + StringBuilder sb = new StringBuilder(stringValue); + if (stringValue.length() == DATETIME_PATTERN.length()) { + sb.append("."); + } + while (sb.toString().length() < DATETIMEV2_PATTERN.length()) { + sb.append(0); + } + return sb.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java new file mode 100644 index 0000000..d82ffbf --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java @@ -0,0 +1,54 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; +import java.sql.SQLException; +import java.util.LinkedHashMap; + +public class OracleSchema extends JdbcSourceSchema { + + public OracleSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return OracleType.toOceanBaseType(fieldType, precision, scale); + } + + @Override + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + // Oracle permits table names to include special characters such as /, + // etc., as in 'A/B'. + // When attempting to fetch column information for `A/B` via JDBC, + // it may throw an ORA-01424 error. + // Hence, we substitute `/` with '_' to address the issue. + return super.getColumnInfo(metaData, databaseName, schemaName, tableName.replace("/", "_")); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java new file mode 100644 index 0000000..dd75bf7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java @@ -0,0 +1,106 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class OracleType { + private static final String VARCHAR2 = "VARCHAR2"; + private static final String NVARCHAR2 = "NVARCHAR2"; + private static final String NUMBER = "NUMBER"; + private static final String FLOAT = "FLOAT"; + private static final String LONG = "LONG"; + private static final String DATE = "DATE"; + private static final String BINARY_FLOAT = "BINARY_FLOAT"; + private static final String BINARY_DOUBLE = "BINARY_DOUBLE"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String INTERVAL = "INTERVAL"; + private static final String RAW = "RAW"; + private static final String LONG_RAW = "LONG RAW"; + private static final String CHAR = "CHAR"; + private static final String NCHAR = "NCHAR"; + private static final String CLOB = "CLOB"; + private static final String NCLOB = "NCLOB"; + private static final String BLOB = "BLOB"; + private static final String BFILE = "BFILE"; + private static final String XMLTYPE = "XMLTYPE"; + + public static String toOceanBaseType(String oracleType, Integer precision, Integer scale) { + oracleType = oracleType.toUpperCase(); + if (oracleType.startsWith(INTERVAL)) { + oracleType = oracleType.substring(0, 8); + } else if (oracleType.startsWith(TIMESTAMP)) { + return String.format("%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale, 6)); + } + switch (oracleType) { + case NUMBER: + if (scale <= 0) { + precision -= scale; + if (precision < 3) { + return OceanBaseType.TINYINT; + } else if (precision < 5) { + return OceanBaseType.SMALLINT; + } else if (precision < 10) { + return OceanBaseType.INT; + } else if (precision < 19) { + return OceanBaseType.BIGINT; + } else { + return OceanBaseType.LARGEINT; + } + } + // scale > 0 + if (precision < scale) { + precision = scale; + } + return precision != null && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.TIMESTAMP, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case FLOAT: + return OceanBaseType.DOUBLE; + case DATE: + // can save date and time with second precision + return OceanBaseType.DATE; + case CHAR: + case VARCHAR2: + case NCHAR: + case NVARCHAR2: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case LONG: + case RAW: + case LONG_RAW: + case INTERVAL: + case BLOB: + case CLOB: + case NCLOB: + case XMLTYPE: + return OceanBaseType.TEXT; + case BFILE: + case BINARY_FLOAT: + case BINARY_DOUBLE: + default: + throw new UnsupportedOperationException("Unsupported Oracle Type: " + oracleType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java new file mode 100644 index 0000000..dc95e39 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java @@ -0,0 +1,247 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.postgres.PostgreSQLSource; +import org.apache.flink.cdc.connectors.postgres.source.PostgresSourceBuilder; +import org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.DECODING_PLUGIN_NAME; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.SLOT_NAME; + +public class PostgresDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(PostgresDatabaseSync.class); + + private static final String JDBC_URL = "jdbc:postgresql://%s:%d/%s?"; + + public PostgresDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("org.postgresql.Driver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class org.postgresql.Driver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(PostgresSourceOptions.HOSTNAME), + config.get(PostgresSourceOptions.PG_PORT), + config.get(PostgresSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(PostgresSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(PostgresSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(PostgresSourceOptions.DATABASE_NAME); + String schemaName = config.get(PostgresSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables( + databaseName, + schemaName, + "%", + new String[] {"TABLE", "PARTITIONED TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new PostgresSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(PostgresSourceOptions.DATABASE_NAME); + String schemaName = config.get(PostgresSourceOptions.SCHEMA_NAME); + String slotName = config.get(SLOT_NAME); + Preconditions.checkNotNull(databaseName, "database-name in postgres is required"); + Preconditions.checkNotNull(schemaName, "schema-name in postgres is required"); + Preconditions.checkNotNull(slotName, "slot.name in postgres is required"); + + String tableName = config.get(PostgresSourceOptions.TABLE_NAME); + String hostname = config.get(PostgresSourceOptions.HOSTNAME); + Integer port = config.get(PostgresSourceOptions.PG_PORT); + String username = config.get(PostgresSourceOptions.USERNAME); + String password = config.get(PostgresSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(PostgresSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(PostgresDateConverter.DEFAULT_PROPS); + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + PostgresSourceBuilder.PostgresIncrementalSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .deserializer(schema) + .slotName(slotName) + .decodingPluginName(config.get(DECODING_PLUGIN_NAME)) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .heartbeatInterval(config.get(HEARTBEAT_INTERVAL)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "Postgres IncrSource"); + } else { + DebeziumSourceFunction postgresSource = + PostgreSQLSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .deserializer(schema) + .slotName(slotName) + .decodingPluginName(config.get(DECODING_PLUGIN_NAME)) + .build(); + return env.addSource(postgresSource, "Postgres Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(PostgresSourceOptions.SCHEMA_NAME); + } + + @Override + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + + if (!initialJdbcUrl.startsWith("?")) { + return super.getJdbcUrlTemplate(initialJdbcUrl, jdbcProperties); + } + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + int recordIndex = 0; + for (Map.Entry entry : jdbcProperties.entrySet()) { + jdbcUrlBuilder.append(entry.getKey()).append("=").append(entry.getValue()); + if (recordIndex < jdbcProperties.size() - 1) { + jdbcUrlBuilder.append("&"); + recordIndex++; + } + } + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java new file mode 100644 index 0000000..fe0152c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java @@ -0,0 +1,133 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class PostgresDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(PostgresDateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.postgres.PostgresDateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } else if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof String) { + return input.toString(); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } else if (input instanceof Instant) { + LocalDateTime ldt = LocalDateTime.ofInstant(((Instant) input), ZoneOffset.UTC); + return timestampFormatter.format(ldt); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java new file mode 100644 index 0000000..d637555 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class PostgresSchema extends JdbcSourceSchema { + + public PostgresSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return PostgresType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java new file mode 100644 index 0000000..1cd42aa --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java @@ -0,0 +1,159 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class PostgresType { + private static final String INT2 = "int2"; + private static final String SMALLSERIAL = "smallserial"; + private static final String INT4 = "int4"; + private static final String SERIAL = "serial"; + private static final String INT8 = "int8"; + private static final String BIGSERIAL = "bigserial"; + private static final String NUMERIC = "numeric"; + private static final String FLOAT4 = "float4"; + private static final String FLOAT8 = "float8"; + private static final String BPCHAR = "bpchar"; + private static final String TIMESTAMP = "timestamp"; + private static final String TIMESTAMPTZ = "timestamptz"; + private static final String DATE = "date"; + private static final String BOOL = "bool"; + private static final String BIT = "bit"; + private static final String POINT = "point"; + private static final String LINE = "line"; + private static final String LSEG = "lseg"; + private static final String BOX = "box"; + private static final String PATH = "path"; + private static final String POLYGON = "polygon"; + private static final String CIRCLE = "circle"; + private static final String VARCHAR = "varchar"; + private static final String TEXT = "text"; + private static final String TIME = "time"; + private static final String TIMETZ = "timetz"; + private static final String INTERVAL = "interval"; + private static final String CIDR = "cidr"; + private static final String INET = "inet"; + private static final String MACADDR = "macaddr"; + private static final String VARBIT = "varbit"; + private static final String UUID = "uuid"; + private static final String BYTEA = "bytea"; + private static final String JSON = "json"; + private static final String JSONB = "jsonb"; + private static final String _INT2 = "_int2"; + private static final String _INT4 = "_int4"; + private static final String _INT8 = "_int8"; + private static final String _FLOAT4 = "_float4"; + private static final String _FLOAT8 = "_float8"; + private static final String _DATE = "_date"; + private static final String _TIMESTAMP = "_timestamp"; + private static final String _BOOL = "_bool"; + private static final String _TEXT = "_text"; + + public static String toOceanBaseType(String postgresType, Integer precision, Integer scale) { + postgresType = postgresType.toLowerCase(); + if (postgresType.startsWith("_")) { + return OceanBaseType.VARCHAR; + } + switch (postgresType) { + case INT2: + case SMALLSERIAL: + return OceanBaseType.TINYINT; + case INT4: + case SERIAL: + return OceanBaseType.INT; + case INT8: + case BIGSERIAL: + return OceanBaseType.BIGINT; + case NUMERIC: + return precision != null && precision > 0 && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.DECIMAL, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case FLOAT4: + return OceanBaseType.FLOAT; + case FLOAT8: + return OceanBaseType.DOUBLE; + case TIMESTAMP: + case TIMESTAMPTZ: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case DATE: + return OceanBaseType.DATE; + case BOOL: + return OceanBaseType.BOOLEAN; + case BIT: + return precision == 1 ? OceanBaseType.BOOLEAN : OceanBaseType.VARCHAR; + case BPCHAR: + case VARCHAR: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case POINT: + case LINE: + case LSEG: + case BOX: + case PATH: + case POLYGON: + case CIRCLE: + case TEXT: + case TIME: + case TIMETZ: + case INTERVAL: + case CIDR: + case INET: + case MACADDR: + case VARBIT: + case UUID: + case BYTEA: + return OceanBaseType.VARCHAR; + case JSON: + case JSONB: + return OceanBaseType.JSONB; + /* Compatible with oceanbase1.2 array type can only be used in dup table, + and then converted to array in the next version + case _BOOL: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.BOOLEAN); + case _INT2: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.TINYINT); + case _INT4: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.INT); + case _INT8: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.BIGINT); + case _FLOAT4: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.FLOAT); + case _FLOAT8: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DOUBLE); + case _TEXT: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.STRING); + case _DATE: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DATE_V2); + case _TIMESTAMP: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DATETIME_V2); + **/ + default: + throw new UnsupportedOperationException( + "Unsupported Postgres Type: " + postgresType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java new file mode 100644 index 0000000..d44dcb7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java @@ -0,0 +1,224 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.flink.cdc.connectors.sqlserver.SqlServerSource; +import org.apache.flink.cdc.connectors.sqlserver.source.SqlServerSourceBuilder; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class SqlServerDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(SqlServerDatabaseSync.class); + private static final String JDBC_URL = "jdbc:sqlserver://%s:%d;database=%s;"; + private static final String PORT = "port"; + + public SqlServerDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.microsoft.sqlserver.jdbc.SQLServerDriver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class com.microsoft.sqlserver.jdbc.SQLServerDriver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(JdbcSourceOptions.HOSTNAME), + config.getInteger(PORT, 1433), + config.get(JdbcSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(JdbcSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(JdbcSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(databaseName, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new SqlServerSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in sqlserver is required"); + Preconditions.checkNotNull(schemaName, "schema-name in sqlserver is required"); + + String tableName = config.get(JdbcSourceOptions.TABLE_NAME); + String hostname = config.get(JdbcSourceOptions.HOSTNAME); + int port = config.getInteger(PORT, 1433); + String username = config.get(JdbcSourceOptions.USERNAME); + String password = config.get(JdbcSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(JdbcSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(SqlServerDateConverter.DEFAULT_PROPS); + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + customConverterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric"); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + SqlServerSourceBuilder.SqlServerIncrementalSource.builder() + .hostname(hostname) + .port(port) + .databaseList(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .startupOptions(startupOptions) + .deserializer(schema) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "SqlServer IncrSource"); + } else { + DebeziumSourceFunction sqlServerSource = + SqlServerSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(sqlServerSource, "SqlServer Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(JdbcSourceOptions.SCHEMA_NAME); + } + + @Override + public String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + jdbcProperties.forEach( + (key, value) -> jdbcUrlBuilder.append(key).append("=").append(value).append(";")); + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java new file mode 100644 index 0000000..b29034c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java @@ -0,0 +1,112 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class SqlServerDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(SqlServerDateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.sqlserver.SqlServerDateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.SMALLDATETIME.equals(sqlType) + || DatabaseSyncConfig.DATETIME.equals(sqlType) + || DatabaseSyncConfig.DATETIME2.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDateTime; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private Object convertDateTime(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } + return null; + } + + private String convertDate(Object input) { + if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java new file mode 100644 index 0000000..9706819 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class SqlServerSchema extends JdbcSourceSchema { + + public SqlServerSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return SqlServerType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java new file mode 100644 index 0000000..3e1b044 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java @@ -0,0 +1,115 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +public class SqlServerType { + private static final String BIT = "bit"; + private static final String TINYINT = "tinyint"; + private static final String SMALLINT = "smallint"; + private static final String INT = "int"; + private static final String BIGINT = "bigint"; + private static final String REAL = "real"; + private static final String FLOAT = "float"; + private static final String MONEY = "money"; + private static final String SMALLMONEY = "smallmoney"; + private static final String DECIMAL = "decimal"; + private static final String NUMERIC = "numeric"; + private static final String DATE = "date"; + private static final String DATETIME = "datetime"; + private static final String DATETIME2 = "datetime2"; + private static final String SMALLDATETIME = "smalldatetime"; + private static final String CHAR = "char"; + private static final String VARCHAR = "varchar"; + private static final String NCHAR = "nchar"; + private static final String NVARCHAR = "nvarchar"; + private static final String TEXT = "text"; + private static final String NTEXT = "ntext"; + private static final String XML = "xml"; + private static final String UNIQUEIDENTIFIER = "uniqueidentifier"; + private static final String TIME = "time"; + private static final String TIMESTAMP = "timestamp"; + private static final String DATETIMEOFFSET = "datetimeoffset"; + private static final String IMAGE = "image"; + private static final String BINARY = "binary"; + private static final String VARBINARY = "varbinary"; + + public static String toOceanBaseType( + String originSqlServerType, Integer precision, Integer scale) { + originSqlServerType = originSqlServerType.toLowerCase(); + // For sqlserver IDENTITY type, such as 'INT IDENTITY' + // originSqlServerType is "int identity", so we only get "int". + String sqlServerType = originSqlServerType.split(" ")[0]; + switch (sqlServerType) { + case BIT: + return OceanBaseType.BOOLEAN; + case TINYINT: + return OceanBaseType.TINYINT; + case SMALLINT: + return OceanBaseType.SMALLINT; + case INT: + return OceanBaseType.INT; + case BIGINT: + return OceanBaseType.BIGINT; + case REAL: + return OceanBaseType.FLOAT; + case FLOAT: + return OceanBaseType.DOUBLE; + case MONEY: + return String.format("%s(%s,%s)", OceanBaseType.DATETIME, 19, 4); + case SMALLMONEY: + return String.format("%s(%s,%s)", OceanBaseType.DATETIME, 10, 4); + case DECIMAL: + case NUMERIC: + return precision != null && precision > 0 && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.DATETIME, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case DATE: + return OceanBaseType.DATE; + case DATETIME: + case DATETIME2: + case SMALLDATETIME: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case CHAR: + case VARCHAR: + case NCHAR: + case NVARCHAR: + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case TEXT: + case NTEXT: + case TIME: + case DATETIMEOFFSET: + case TIMESTAMP: + case UNIQUEIDENTIFIER: + case BINARY: + case VARBINARY: + case XML: + return OceanBaseType.TEXT; + default: + throw new UnsupportedOperationException( + "Unsupported SqlServer Type: " + sqlServerType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java new file mode 100644 index 0000000..e196ca9 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java @@ -0,0 +1,133 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.utils; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseSchemaFactory; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; + +import org.apache.flink.util.function.SupplierWithException; + +import org.apache.commons.compress.utils.Lists; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Predicate; + +import static org.apache.flink.util.Preconditions.checkArgument; + +public class OceanBaseToolsJdbcUtils extends OceanBaseJdbcUtils { + private static final List builtinDatabases = + Collections.singletonList("information_schema"); + + public static List listDatabases( + SupplierWithException connectionSupplier) { + return extractColumnValuesBySQL( + connectionSupplier, + "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA`;", + 1, + dbName -> !builtinDatabases.contains(dbName)); + } + + public static boolean databaseExists( + String database, SupplierWithException connectionSupplier) { + checkArgument(!org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly(database)); + return listDatabases(connectionSupplier).contains(database); + } + + public static void createDatabase( + String database, SupplierWithException connectionSupplier) { + execute(connectionSupplier, String.format("CREATE DATABASE IF NOT EXISTS %s", database)); + } + + public static boolean tableExists( + String database, + String table, + SupplierWithException connectionSupplier) { + return databaseExists(database, connectionSupplier) + && listTables(database, connectionSupplier).contains(table); + } + + public static List listTables( + String databaseName, + SupplierWithException connectionSupplier) { + if (!databaseExists(databaseName, connectionSupplier)) { + throw new RuntimeException("database" + databaseName + " is not exists"); + } + return extractColumnValuesBySQL( + connectionSupplier, + "SELECT TABLE_NAME FROM information_schema.`TABLES` WHERE TABLE_SCHEMA = ?", + 1, + null, + databaseName); + } + + public static void createTable( + TableSchema schema, + SupplierWithException connectionSupplier) { + String ddl = buildCreateTableDDL(schema); + execute(connectionSupplier, ddl); + } + + public static void execute( + SupplierWithException connectionSupplier, String sql) { + try (Connection connection = connectionSupplier.get(); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } catch (Exception e) { + throw new RuntimeException( + String.format("SQL query could not be executed: %s", sql), e); + } + } + + public static List extractColumnValuesBySQL( + SupplierWithException connectionSupplier, + String sql, + int columnIndex, + Predicate filterFunc, + Object... params) { + List columnValues = Lists.newArrayList(); + try (Connection connection = connectionSupplier.get(); + PreparedStatement ps = connection.prepareStatement(sql)) { + if (Objects.nonNull(params) && params.length > 0) { + for (int i = 0; i < params.length; i++) { + ps.setObject(i + 1, params[i]); + } + } + try (ResultSet rs = ps.executeQuery()) { + while (rs.next()) { + String columnValue = rs.getString(columnIndex); + if (filterFunc == null || filterFunc.test(columnValue)) { + columnValues.add(columnValue); + } + } + } + return columnValues; + } catch (Exception e) { + throw new RuntimeException( + String.format("The following SQL query could not be executed: %s", sql), e); + } + } + + public static String buildCreateTableDDL(TableSchema schema) { + return OceanBaseSchemaFactory.generateCreateTableDDL(schema); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java new file mode 100644 index 0000000..fa0d2eb --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java @@ -0,0 +1,222 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.OceanBaseMySQLTestBase; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.mysql.MysqlDatabaseSync; + +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.shaded.org.apache.commons.io.IOUtils; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +public class CdcMysqlSyncDatabaseCase extends OceanBaseMySQLTestBase { + private static final Logger LOG = LoggerFactory.getLogger(CdcMysqlSyncDatabaseCase.class); + + private static final String MYSQL_HOST = "localhost"; + private static final Integer MYSQL_PORT = 3306; + private static final String MYSQL_USER_NAME = "root"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; + private static final String MYSQL_DATABASE = "mysql_cdc"; + private static final String MYSQL_TABLE_NAME = "test_history_text"; + static StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + @BeforeClass + public static void setup() { + CONFIG_SERVER.withLogConsumer(new Slf4jLogConsumer(LOG)).start(); + CONTAINER + .withEnv("OB_CONFIGSERVER_ADDRESS", getConfigServerAddress()) + .withLogConsumer(new Slf4jLogConsumer(LOG)) + .start(); + MYSQL_CONTAINER.start(); + } + + @AfterClass + public static void tearDown() { + Stream.of(MYSQL_CONTAINER).forEach(GenericContainer::stop); + } + + private static final MySqlContainer MYSQL_CONTAINER = + new MySqlContainer() + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("sql/cdc.sql") + // .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withExposedPorts(MYSQL_PORT) + .withDatabaseName(MYSQL_DATABASE) + .withPassword(MYSQL_USER_PASSWORD) + .withEnv("TZ", "Asia/Shanghai") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("mysql-docker-image"))); + + @Test + public void testCdcMysqlSyncOceanBase() throws Exception { + extractedCdcSync(); + checkResult(); + } + + private static void extractedCdcSync() throws Exception { + // env.setParallelism(1); + Map flinkMap = new HashMap<>(); + flinkMap.put("execution.checkpointing.interval", "10s"); + flinkMap.put("pipeline.operator-chaining", "false"); + flinkMap.put("parallelism.default", "1"); + + Configuration configuration = Configuration.fromMap(flinkMap); + env.configure(configuration); + + String tablePrefix = ""; + String tableSuffix = ""; + Map mysqlConfig = new HashMap<>(); + mysqlConfig.put(MySqlSourceOptions.DATABASE_NAME.key(), MYSQL_DATABASE); + mysqlConfig.put(MySqlSourceOptions.HOSTNAME.key(), MYSQL_HOST); + mysqlConfig.put( + MySqlSourceOptions.PORT.key(), + String.valueOf(MYSQL_CONTAINER.getMappedPort(MYSQL_PORT))); + mysqlConfig.put(MySqlSourceOptions.USERNAME.key(), MYSQL_USER_NAME); + mysqlConfig.put(MySqlSourceOptions.PASSWORD.key(), MYSQL_USER_PASSWORD); + // add jdbc properties for MySQL + mysqlConfig.put("jdbc.properties.use_ssl", "false"); + Configuration config = Configuration.fromMap(mysqlConfig); + + Map sinkConfig = new HashMap<>(); + sinkConfig.put(OceanBaseConnectorOptions.USERNAME.key(), CONTAINER.getUsername()); + sinkConfig.put(OceanBaseConnectorOptions.PASSWORD.key(), CONTAINER.getPassword()); + sinkConfig.put(OceanBaseConnectorOptions.URL.key(), CONTAINER.getJdbcUrl()); + sinkConfig.put("sink.enable-delete", "false"); + Configuration sinkConf = Configuration.fromMap(sinkConfig); + + String includingTables = "test.*"; + String excludingTables = ""; + boolean ignoreDefaultValue = false; + boolean ignoreIncompatible = false; + DatabaseSync databaseSync = new MysqlDatabaseSync(); + databaseSync + .setEnv(env) + .setDatabase(MYSQL_DATABASE) + .setConfig(config) + .setTablePrefix(tablePrefix) + .setTableSuffix(tableSuffix) + .setIncludingTables(includingTables) + .setExcludingTables(excludingTables) + .setIgnoreDefaultValue(ignoreDefaultValue) + .setSinkConfig(sinkConf) + .setCreateTableOnly(false) + .create(); + databaseSync.build(); + env.executeAsync(String.format("MySQL-Doris Database Sync: %s", MYSQL_DATABASE)); + checkResult(); + env.close(); + } + + static void checkResult() { + String sourceSql = String.format("select * from %s order by 1", MYSQL_TABLE_NAME); + String sinkSql = String.format("select * from %s order by 1", MYSQL_TABLE_NAME); + try (Statement sourceStatement = + getConnection( + getJdbcUrl( + MYSQL_HOST, + MYSQL_CONTAINER.getMappedPort(MYSQL_PORT), + MYSQL_DATABASE), + MYSQL_USER_NAME, + MYSQL_USER_PASSWORD) + .createStatement( + ResultSet.TYPE_SCROLL_INSENSITIVE, + ResultSet.CONCUR_READ_ONLY); + Statement sinkStatement = + getConnection( + CONTAINER.getJdbcUrl(), + CONTAINER.getUsername(), + CONTAINER.getPassword()) + .createStatement( + ResultSet.TYPE_SCROLL_INSENSITIVE, + ResultSet.CONCUR_READ_ONLY); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql)) { + Assertions.assertEquals( + sourceResultSet.getMetaData().getColumnCount(), + sinkResultSet.getMetaData().getColumnCount()); + while (sourceResultSet.next()) { + if (sinkResultSet.next()) { + for (String column : getFieldNames()) { + Object source = sourceResultSet.getObject(column); + Object sink = sinkResultSet.getObject(column); + if (!Objects.deepEquals(source, sink)) { + InputStream sourceAsciiStream = sourceResultSet.getBinaryStream(column); + InputStream sinkAsciiStream = sinkResultSet.getBinaryStream(column); + String sourceValue = + IOUtils.toString(sourceAsciiStream, StandardCharsets.UTF_8); + String sinkValue = + IOUtils.toString(sinkAsciiStream, StandardCharsets.UTF_8); + Assertions.assertEquals(sourceValue, sinkValue); + } + } + } + } + sourceResultSet.last(); + sinkResultSet.last(); + } catch (Exception e) { + throw new RuntimeException("Compare result error", e); + } + } + + static String[] getFieldNames() { + return new String[] { + "itemid", "clock", "value", "ns", + }; + } + + public static Connection getConnection(String jdbcUrl, String userName, String password) + throws SQLException { + return DriverManager.getConnection(jdbcUrl, userName, password); + } + + public static String getJdbcUrl(String host, Integer port, String schema) { + return "jdbc:mysql://" + + host + + ":" + + port + + "/" + + schema + + "?useUnicode=true&characterEncoding=UTF-8&useSSL=false"; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java new file mode 100644 index 0000000..6c7dce0 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java @@ -0,0 +1,94 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.tools.cdc.CdcTools; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.api.java.utils.MultipleParameterTool; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +public class CdcToolsTest { + + @Test + public void getConfigMapTest() { + MultipleParameterTool params = + MultipleParameterTool.fromArgs( + new String[] { + "--sink-conf", + "password=password", + "--sink-conf", + "username=username", + "--sink-conf", + "url=jdbc:mysql://127.0.0.1:2881 " + }); + Map sinkConf = CdcTools.getConfigMap(params, DatabaseSyncConfig.SINK_CONF); + + Map excepted = new HashMap<>(); + excepted.put("password", "password"); + excepted.put("username", "username"); + excepted.put("url", "jdbc:mysql://127.0.0.1:2881"); + Assert.assertEquals(sinkConf, excepted); + + Map mysqlConf = + CdcTools.getConfigMap(params, DatabaseSyncConfig.MYSQL_CONF); + Assert.assertNull(mysqlConf); + } + + @Test + public void testGetConfigMap() { + Map> config = new HashMap<>(); + config.put( + DatabaseSyncConfig.MYSQL_CONF, Arrays.asList(" hostname=127.0.0.1", " port=3306")); + config.put( + DatabaseSyncConfig.POSTGRES_CONF, + Arrays.asList("hostname=127.0.0.1 ", "port=5432 ")); + MultipleParameterTool parameter = MultipleParameterTool.fromMultiMap(config); + Map mysqlConfigMap = + CdcTools.getConfigMap(parameter, DatabaseSyncConfig.MYSQL_CONF); + Map postGresConfigMap = + CdcTools.getConfigMap(parameter, DatabaseSyncConfig.POSTGRES_CONF); + + Set mysqlKeyConf = new HashSet<>(Arrays.asList("hostname", "port")); + Set mysqlValueConf = new HashSet<>(Arrays.asList("127.0.0.1", "3306")); + assertEquals(mysqlConfigMap, mysqlKeyConf, mysqlValueConf); + + Set postgresKeyConf = new HashSet<>(Arrays.asList("hostname", "port")); + Set postgresValueConf = new HashSet<>(Arrays.asList("127.0.0.1", "5432")); + assertEquals(postGresConfigMap, postgresKeyConf, postgresValueConf); + } + + private void assertEquals( + Map actualMap, Set keyConf, Set valueConf) { + for (Entry entry : actualMap.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + Assert.assertTrue(keyConf.contains(key)); + Assert.assertTrue(valueConf.contains(value)); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java new file mode 100644 index 0000000..829fee5 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java @@ -0,0 +1,174 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import org.testcontainers.containers.ContainerLaunchException; +import org.testcontainers.containers.JdbcDatabaseContainer; +import org.testcontainers.utility.DockerImageName; + +import java.util.HashSet; +import java.util.Set; + +/** + * Docker container for MySQL. The difference between this class and {@link + * org.testcontainers.containers.MySqlContainer} is that TC MySQLContainer has problems when + * overriding mysql conf file, i.e. my.cnf. + */ +@SuppressWarnings("MagicNumber") +public class MySqlContainer extends JdbcDatabaseContainer { + + public static final String IMAGE = "mysql"; + public static final Integer MYSQL_PORT = 3306; + + private static final String MY_CNF_CONFIG_OVERRIDE_PARAM_NAME = "MY_CNF"; + private static final String SETUP_SQL_PARAM_NAME = "SETUP_SQL"; + private static final String MYSQL_ROOT_USER = "root"; + + private String databaseName = "test"; + private String username = "test"; + private String password = "test"; + + public MySqlContainer() { + super(DockerImageName.parse(IMAGE + ":" + "8.0.20")); + addExposedPort(MYSQL_PORT); + } + + @Override + protected Set getLivenessCheckPorts() { + return new HashSet<>(getMappedPort(MYSQL_PORT)); + } + + @Override + protected void configure() { + optionallyMapResourceParameterAsVolume( + MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, "/etc/mysql/", "mysql-default-conf"); + + if (parameters.containsKey(SETUP_SQL_PARAM_NAME)) { + optionallyMapResourceParameterAsVolume( + SETUP_SQL_PARAM_NAME, "/docker-entrypoint-initdb.d/", "N/A"); + } + + addEnv("MYSQL_DATABASE", databaseName); + addEnv("MYSQL_USER", username); + if (password != null && !password.isEmpty()) { + addEnv("MYSQL_PASSWORD", password); + addEnv("MYSQL_ROOT_PASSWORD", password); + } else { + throw new ContainerLaunchException( + "Empty password can be used only with the root user"); + } + setStartupAttempts(3); + } + + @Override + public String getDriverClassName() { + try { + Class.forName("com.mysql.cj.jdbc.Driver"); + return "com.mysql.cj.jdbc.Driver"; + } catch (ClassNotFoundException e) { + return "com.mysql.jdbc.Driver"; + } + } + + public String getJdbcUrl(String databaseName) { + String additionalUrlParams = constructUrlParameters("?", "&"); + return "jdbc:mysql://" + + getHost() + + ":" + + getDatabasePort() + + "/" + + databaseName + + additionalUrlParams; + } + + public void setDatabaseName(String databaseName) { + this.databaseName = databaseName; + } + + @Override + public String getJdbcUrl() { + return getJdbcUrl(databaseName); + } + + public int getDatabasePort() { + return getMappedPort(MYSQL_PORT); + } + + @Override + protected String constructUrlForConnection(String queryString) { + String url = super.constructUrlForConnection(queryString); + + if (!url.contains("useSSL=")) { + String separator = url.contains("?") ? "&" : "?"; + url = url + separator + "useSSL=false"; + } + + if (!url.contains("allowPublicKeyRetrieval=")) { + url = url + "&allowPublicKeyRetrieval=true"; + } + + return url; + } + + @Override + public String getDatabaseName() { + return databaseName; + } + + @Override + public String getUsername() { + return username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + protected String getTestQueryString() { + return "SELECT 1"; + } + + public MySqlContainer withConfigurationOverride(String s) { + parameters.put(MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, s); + return this; + } + + public MySqlContainer withSetupSQL(String sqlPath) { + parameters.put(SETUP_SQL_PARAM_NAME, sqlPath); + return this; + } + + @Override + public MySqlContainer withDatabaseName(final String databaseName) { + this.databaseName = databaseName; + return this; + } + + @Override + public MySqlContainer withUsername(final String username) { + this.username = username; + return this; + } + + @Override + public MySqlContainer withPassword(final String password) { + this.password = password; + return this; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java new file mode 100644 index 0000000..4159c5a --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java @@ -0,0 +1,186 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.table.OceanBaseJsonSerializationSchema; +import com.oceanbase.connector.flink.table.SerializationRuntimeConverter; +import com.oceanbase.connector.flink.table.TableInfo; + +import org.apache.flink.table.data.*; +import org.apache.flink.table.types.logical.*; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.time.*; +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class OceanBaseJsonSerializationSchemaTest { + + @Test + void testCreateNotNullConverter() { + OceanBaseJsonSerializationSchemaConverterTest schema = + new OceanBaseJsonSerializationSchemaConverterTest(null); + // Test Boolean + assertEquals(true, schema.createNotNullConverter(new BooleanType()).convert(true)); + + // Test TinyInt + assertEquals( + (byte) 123, schema.createNotNullConverter(new TinyIntType()).convert((byte) 123)); + + // Test SmallInt + assertEquals( + (short) 12345, + schema.createNotNullConverter(new SmallIntType()).convert((short) 12345)); + + // Test Integer + assertEquals(123456789, schema.createNotNullConverter(new IntType()).convert(123456789)); + + // Test BigInt + assertEquals( + 123456789012345L, + schema.createNotNullConverter(new BigIntType()).convert(123456789012345L)); + + // Test Float + assertEquals(123.45f, schema.createNotNullConverter(new FloatType()).convert(123.45f)); + + // Test Double + assertEquals( + 123456.789, schema.createNotNullConverter(new DoubleType()).convert(123456.789)); + + // Test Binary + byte[] binaryData = "testBinary".getBytes(); + assertEquals( + binaryData, schema.createNotNullConverter(new BinaryType()).convert(binaryData)); + + // Test VarBinary + byte[] varBinaryData = "testVarBinary".getBytes(); + assertEquals( + varBinaryData, + schema.createNotNullConverter(new VarBinaryType(50)).convert(varBinaryData)); + + // Test Char + assertEquals("testChar", schema.createNotNullConverter(new CharType()).convert("testChar")); + + // Test Varchar + assertEquals( + "testVarchar", + schema.createNotNullConverter(new VarCharType()).convert("testVarchar")); + + LocalDate testDate = LocalDate.of(2024, 1, 1); + int daysSinceEpoch = (int) testDate.toEpochDay(); + Date expectedDate = Date.valueOf(testDate); + assertEquals( + expectedDate, + schema.createNotNullConverter(new DateType()).convert(daysSinceEpoch)); + + // Test TimeWithoutTimeZone + Time time = Time.valueOf("12:34:56"); + assertEquals( + time, + schema.createNotNullConverter(new TimeType()) + .convert(45296000)); // 45296000 ms equals 12:34:56 + + // Test TimestampWithoutTimeZone + TimestampData timestampData = + TimestampData.fromLocalDateTime(LocalDateTime.of(2024, 1, 1, 12, 34, 56)); + assertEquals( + timestampData.toTimestamp(), + schema.createNotNullConverter(new TimestampType()).convert(timestampData)); + + // Test TimestampWithTimeZone + TimestampData timestampWithTZ = + TimestampData.fromInstant(Instant.parse("2024-01-01T12:34:56Z")); + assertEquals( + "2024-01-01T12:34:56Z", + schema.createNotNullConverter(new ZonedTimestampType()).convert(timestampWithTZ)); + + // Test TimestampWithLocalTimeZone + TimestampData timestampWithLocalTZ = + TimestampData.fromInstant(Instant.parse("2024-01-01T12:34:56Z")); + assertEquals( + "2024-01-01T20:34:56+08:00[Asia/Shanghai]", + schema.createNotNullConverter(new LocalZonedTimestampType()) + .convert(timestampWithLocalTZ)); + + // Test Decimal + DecimalData decimalData = DecimalData.fromBigDecimal(new BigDecimal("123456.789"), 9, 3); + assertEquals( + new BigDecimal("123456.789"), + schema.createNotNullConverter(new DecimalType(9, 3)).convert(decimalData)); + + // Test Array + ArrayData arrayData = + new GenericArrayData( + new StringData[] { + StringData.fromString("a"), + StringData.fromString("b"), + StringData.fromString("c") + }); + String expectedArrayString = "a,b,c"; + String actualArrayString = + (String) + schema.createNotNullConverter(new ArrayType(new VarCharType())) + .convert(arrayData); + assertEquals(expectedArrayString, actualArrayString); + + // Test Map + Map map = new LinkedHashMap<>(); + map.put(StringData.fromString("key1"), StringData.fromString("value1")); + map.put(StringData.fromString("key2"), StringData.fromString("value2")); + MapData mapData = new GenericMapData(map); + // Assume there is a toString method that formats MapData correctly, or you can compare each + // element + String expectedMapString = "{\"key1\":\"value1\",\"key2\":\"value2\"}"; + String actualMapString = + (String) + schema.createNotNullConverter( + new MapType(new VarCharType(), new VarCharType())) + .convert(mapData); + assertEquals(expectedMapString, actualMapString); + + // Test Row + RowType rowType = + new RowType( + java.util.Arrays.asList( + new RowType.RowField("field1", new VarCharType()), + new RowType.RowField("field2", new IntType()), + new RowType.RowField("field3", new BooleanType()))); + RowData rowData = GenericRowData.of(StringData.fromString("field1"), 123, true); + assertEquals( + "{\"field1\":\"field1\",\"field2\":123,\"field3\":true}", + schema.createNotNullConverter(rowType).convert(rowData)); + } + + // Create a subclass to expose the protected createNotNullConverter method + private static class OceanBaseJsonSerializationSchemaConverterTest + extends OceanBaseJsonSerializationSchema { + + public OceanBaseJsonSerializationSchemaConverterTest(TableInfo tableInfo) { + super(tableInfo); + } + + @Override + public SerializationRuntimeConverter createNotNullConverter(LogicalType type) { + return super.createNotNullConverter(type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf b/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 0000000..a390897 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties b/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000..68c8b84 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties @@ -0,0 +1,21 @@ +# Copyright 2024 OceanBase. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +rootLogger.level = INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql b/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql new file mode 100644 index 0000000..dfb1378 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql @@ -0,0 +1,11 @@ + +CREATE TABLE test_history_text ( + itemid bigint NOT NULL, + clock integer DEFAULT '0' NOT NULL, + value text NOT NULL, + ns integer DEFAULT '0' NOT NULL, + PRIMARY KEY (itemid,clock,ns) +); +INSERT INTO test_history_text (itemid,clock,value,ns) VALUES + (1,21131,'ces1',21321), + (2,21321,'ces2',12321); diff --git a/pom.xml b/pom.xml index 4a432c8..2efcc20 100644 --- a/pom.xml +++ b/pom.xml @@ -37,12 +37,16 @@ under the License. flink-sql-connector-oceanbase flink-sql-connector-oceanbase-directload flink-connector-oceanbase-e2e-tests + flink-connector-oceanbase-tools-cdc 1.5-SNAPSHOT 1.18.0 2.12 + 19.3.0.0 + 3.1.1 + 1.17.6 com.oceanbase.connector.flink.shaded
In the case where oceanbase schema needs to be created, it is best to create it through this + * factory + */ +public class OceanBaseSchemaFactory { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseSchemaFactory.class); + private static final OceanBaseDialect dialect = new OceanBaseMySQLDialect(); + + public static TableSchema createTableSchema( + String database, + String table, + Map columnFields, + List pkKeys, + String tableComment) { + TableSchema tableSchema = new TableSchema(); + tableSchema.setDatabase(database); + tableSchema.setTable(table); + tableSchema.setFields(columnFields); + tableSchema.setKeys(buildKeys(pkKeys, columnFields)); + tableSchema.setTableComment(tableComment); + return tableSchema; + } + + private static List buildDistributeKeys( + List primaryKeys, Map fields) { + return buildKeys(primaryKeys, fields); + } + + /** + * Theoretically, the duplicate table of oceanbase does not need to distinguish the key column, + * but in the actual table creation statement, the key column will be automatically added. So if + * it is a duplicate table, primaryKeys is empty, and we uniformly take the first field as the + * key. + */ + private static List buildKeys( + List primaryKeys, Map fields) { + if (CollectionUtils.isNotEmpty(primaryKeys)) { + return primaryKeys; + } + if (!fields.isEmpty()) { + Entry firstField = fields.entrySet().iterator().next(); + return Collections.singletonList(firstField.getKey()); + } + return new ArrayList<>(); + } + + public static String generateCreateTableDDL(TableSchema schema) { + StringBuilder sb = new StringBuilder("CREATE TABLE IF NOT EXISTS "); + sb.append(dialect.quoteIdentifier(schema.getDatabase())) + .append(".") + .append(dialect.quoteIdentifier(schema.getTable())) + .append(" ("); + + Map fields = schema.getFields(); + List keys = schema.getKeys(); + + // Append fields + for (Map.Entry entry : fields.entrySet()) { + FieldSchema field = entry.getValue(); + buildColumn(sb, field, keys.contains(entry.getKey())); + } + + sb = sb.deleteCharAt(sb.length() - 1); + // Append primary key constraint + if (!keys.isEmpty()) { + sb.append("PRIMARY KEY (") + .append( + keys.stream() + .map(dialect::quoteIdentifier) + .collect(Collectors.joining(","))) + .append(")"); + } + sb.append(")"); + + // Append table comment + if (schema.getTableComment() != null && !schema.getTableComment().trim().isEmpty()) { + sb.append(" COMMENT='").append(quoteComment(schema.getTableComment())).append("'"); + } + sb.append(";"); + + LOG.info("Generated DDL: {}", sb); + return sb.toString(); + } + + private static void buildColumn(StringBuilder sb, FieldSchema field, boolean isKey) { + sb.append(dialect.quoteIdentifier(field.getName())) + .append(" ") + .append(field.getTypeString()); + + if (!isKey && field.getNullable()) { + sb.append(" NULL"); + } else { + sb.append(" NOT NULL"); + } + + if (field.getDefaultValue() != null) { + sb.append(" DEFAULT ").append(quoteDefaultValue(field.getDefaultValue())); + } + + if (field.getComment() != null && !field.getComment().trim().isEmpty()) { + sb.append(" COMMENT '").append(quoteComment(field.getComment())).append("'"); + } + + sb.append(", "); + } + + public static String quoteDefaultValue(String defaultValue) { + // DEFAULT current_timestamp not need quote + if (defaultValue.equalsIgnoreCase("current_timestamp")) { + return defaultValue; + } + return "'" + defaultValue + "'"; + } + + public static String quoteComment(String comment) { + if (comment == null) { + return ""; + } else { + return comment.replaceAll("'", "\\\\'"); + } + } + + public static String quoteTableIdentifier(String tableIdentifier) { + String[] dbTable = tableIdentifier.split("\\."); + Preconditions.checkArgument(dbTable.length == 2); + return dialect.quoteIdentifier(dbTable[0]) + "." + dialect.quoteIdentifier(dbTable[1]); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java new file mode 100644 index 0000000..0827c6e --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseType.java @@ -0,0 +1,41 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +public class OceanBaseType { + public static final String BOOLEAN = "BOOLEAN"; + public static final String TINYINT = "TINYINT"; + public static final String SMALLINT = "SMALLINT"; + public static final String INT = "INT"; + public static final String BIGINT = "BIGINT"; + public static final String LARGEINT = "LARGEINT"; + public static final String FLOAT = "FLOAT"; + public static final String DOUBLE = "DOUBLE"; + public static final String TIMESTAMP = "TIMESTAMP"; + public static final String DECIMAL = "DECIMAL"; + public static final String DATE = "DATE"; + public static final String DATETIME = "DATETIME"; + public static final String CHAR = "CHAR"; + public static final String VARCHAR = "VARCHAR"; + public static final String TEXT = "TEXT"; + public static final String ARRAY = "ARRAY"; + public static final String JSONB = "JSONB"; + public static final String JSON = "JSON"; + public static final String MAP = "MAP"; + public static final String STRUCT = "STRUCT"; + public static final String VARIANT = "VARIANT"; +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java new file mode 100644 index 0000000..83c08e7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/OceanBaseTypeMapper.java @@ -0,0 +1,187 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; + +import java.sql.Types; + +import static com.alibaba.druid.sql.ast.SQLDataType.Constants.BOOLEAN; +import static com.oceanbase.connector.flink.tools.catalog.OceanBaseType.*; + +public class OceanBaseTypeMapper { + + /** Max size of char type of OceanBase. */ + public static final int MAX_CHAR_SIZE = 256; + + /** Max size of varchar type of OceanBase. */ + public static final int MAX_VARCHAR_SIZE = 262144; + /* Max precision of datetime type of OceanBase. */ + public static final int MAX_SUPPORTED_DATE_TIME_PRECISION = 6; + + public static LogicalType convertToLogicalType(int jdbcType, int precision, int scale) { + switch (jdbcType) { + case Types.INTEGER: + return DataTypes.INT().getLogicalType(); + case Types.BIGINT: + return DataTypes.BIGINT().getLogicalType(); + case Types.DOUBLE: + return DataTypes.DOUBLE().getLogicalType(); + case Types.FLOAT: + case Types.REAL: + return DataTypes.FLOAT().getLogicalType(); + case Types.LONGVARCHAR: + case Types.VARCHAR: + if (precision > 0) { + return DataTypes.STRING().getLogicalType(); + } else { + return DataTypes.STRING().getLogicalType(); + } + case Types.CHAR: + if (precision > 0) { + return DataTypes.CHAR(precision).getLogicalType(); + } else { + return DataTypes.STRING().getLogicalType(); + } + case Types.TIMESTAMP: + if (precision > 0 && precision <= 3) { + return DataTypes.TIMESTAMP(precision).getLogicalType(); + } else { + return DataTypes.TIMESTAMP(3).getLogicalType(); + } + case Types.DATE: + return DataTypes.DATE().getLogicalType(); + case Types.TIME: + if (precision > 0 && precision <= 3) { + return DataTypes.TIME(precision).getLogicalType(); + } else { + return DataTypes.TIME(0).getLogicalType(); + } + case Types.DECIMAL: + case Types.NUMERIC: + if (precision > 0 && precision <= 38 && scale >= 0) { + return DataTypes.DECIMAL(precision, scale).getLogicalType(); + } else { + return DataTypes.DECIMAL(10, 0).getLogicalType(); + } + case Types.BOOLEAN: + case Types.BIT: + return DataTypes.BOOLEAN().getLogicalType(); + case Types.TINYINT: + return DataTypes.TINYINT().getLogicalType(); + case Types.SMALLINT: + return DataTypes.SMALLINT().getLogicalType(); + case Types.BLOB: + return DataTypes.BYTES().getLogicalType(); + case Types.CLOB: + return DataTypes.STRING().getLogicalType(); + case Types.BINARY: + if (precision > 0) { + return DataTypes.BINARY(precision).getLogicalType(); + } else { + return DataTypes.BYTES().getLogicalType(); + } + case Types.LONGVARBINARY: + case Types.VARBINARY: + if (precision > 0) { + return DataTypes.VARBINARY(precision).getLogicalType(); + } else { + return DataTypes.BYTES().getLogicalType(); + } + default: + throw new IllegalArgumentException("Unsupported JDBC type: " + jdbcType); + } + } + + public static String toOceanBaseType(DataType flinkType) { + LogicalType logicalType = flinkType.getLogicalType(); + return logicalType.accept(new LogicalTypeVisitor(logicalType)); + } + + private static class LogicalTypeVisitor extends LogicalTypeDefaultVisitor { + private final LogicalType type; + + LogicalTypeVisitor(LogicalType type) { + this.type = type; + } + + @Override + public String visit(CharType charType) { + long length = charType.getLength() * 3L; + if (length <= MAX_CHAR_SIZE) { + return String.format("%s(%s)", OceanBaseType.CHAR, length); + } else { + return visit(new VarCharType(charType.getLength())); + } + } + + @Override + public String visit(BooleanType booleanType) { + return BOOLEAN; + } + + @Override + public String visit(TinyIntType tinyIntType) { + return TINYINT; + } + + @Override + public String visit(SmallIntType smallIntType) { + return SMALLINT; + } + + @Override + public String visit(IntType intType) { + return INT; + } + + @Override + public String visit(BigIntType bigIntType) { + return BIGINT; + } + + @Override + public String visit(FloatType floatType) { + return FLOAT; + } + + @Override + public String visit(DoubleType doubleType) { + return DOUBLE; + } + + @Override + protected String defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support converting type %s to OceanBase type yet.", + type.toString())); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java new file mode 100644 index 0000000..818898a --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/TableSchema.java @@ -0,0 +1,100 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TableSchema { + private String database; + private String table; + private String tableComment; + private Map fields; + private List keys = new ArrayList<>(); + private Map properties = new HashMap<>(); + + public String getDatabase() { + return database; + } + + public String getTable() { + return table; + } + + public String getTableComment() { + return tableComment; + } + + public Map getFields() { + return fields; + } + + public List getKeys() { + return keys; + } + + public Map getProperties() { + return properties; + } + + public void setDatabase(String database) { + this.database = database; + } + + public void setTable(String table) { + this.table = table; + } + + public void setTableComment(String tableComment) { + this.tableComment = tableComment; + } + + public void setFields(Map fields) { + this.fields = fields; + } + + public void setKeys(List keys) { + this.keys = keys; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + @Override + public String toString() { + return "TableSchema{" + + "database='" + + database + + '\'' + + ", table='" + + table + + '\'' + + ", tableComment='" + + tableComment + + '\'' + + ", fields=" + + fields + + ", keys=" + + String.join(",", keys) + + ", properties=" + + properties + + '}'; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java new file mode 100644 index 0000000..073cc08 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/catalog/oceanBaseSinkOperate.java @@ -0,0 +1,124 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.catalog; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; +import com.oceanbase.connector.flink.dialect.OceanBaseMySQLDialect; +import com.oceanbase.connector.flink.sink.OceanBaseRecordFlusher; +import com.oceanbase.connector.flink.sink.OceanBaseSink; +import com.oceanbase.connector.flink.table.DataChangeRecord; +import com.oceanbase.connector.flink.table.OceanBaseJsonSerializationSchema; +import com.oceanbase.connector.flink.table.TableId; +import com.oceanbase.connector.flink.table.TableInfo; + +import org.apache.flink.annotation.Public; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** OceanBase Sink Operate. */ +@Public +public class oceanBaseSinkOperate implements Serializable { + private static final long serialVersionUID = 1L; + protected Configuration sinkConfig; + private final OceanBaseConnectorOptions connectorOptions; + + public oceanBaseSinkOperate(Configuration cdcSinkConfig) { + sinkConfig = cdcSinkConfig; + this.connectorOptions = getOceanBaseConnectorOptions(); + } + + @Deprecated + public static String quoteDefaultValue(String defaultValue) { + return OceanBaseSchemaFactory.quoteDefaultValue(defaultValue); + } + + @Deprecated + public static String quoteComment(String comment) { + return OceanBaseSchemaFactory.quoteComment(comment); + } + + @Deprecated + public static String quoteTableIdentifier(String tableIdentifier) { + return OceanBaseSchemaFactory.quoteTableIdentifier(tableIdentifier); + } + + public OceanBaseSink createGenericOceanBaseSink(String schemaName, String tableName) + throws Exception { + + List columnNames = new ArrayList<>(); + List pkColumnNames = new ArrayList<>(); + List columnTypes = new ArrayList<>(); + OceanBaseMySQLDialect dialect = new OceanBaseMySQLDialect(); + try (OceanBaseConnectionProvider connectionProvider = + new OceanBaseConnectionProvider(connectorOptions)) { + Connection connection = connectionProvider.getConnection(); + DatabaseMetaData metaData = connection.getMetaData(); + try (ResultSet columns = metaData.getColumns(schemaName, schemaName, tableName, null)) { + while (columns.next()) { + String columnName = columns.getString("COLUMN_NAME"); + int dataType = columns.getInt("DATA_TYPE"); + int precision = columns.getInt("COLUMN_SIZE"); + int scale = columns.getInt("DECIMAL_DIGITS"); + columnNames.add(columnName); + columnTypes.add( + OceanBaseTypeMapper.convertToLogicalType(dataType, precision, scale)); + } + } + + try (ResultSet primaryKeys = + metaData.getPrimaryKeys(schemaName, schemaName, tableName)) { + while (primaryKeys.next()) { + String pkColumnName = primaryKeys.getString("COLUMN_NAME"); + pkColumnNames.add(pkColumnName); + } + } + } + TableInfo tableInfo = + new TableInfo( + new TableId(dialect::getFullTableName, schemaName, tableName), + pkColumnNames, + columnNames, + columnTypes, + null); + + return new OceanBaseSink<>( + connectorOptions, + null, + new OceanBaseJsonSerializationSchema(tableInfo), + DataChangeRecord.KeyExtractor.simple(), + new OceanBaseRecordFlusher(connectorOptions)); + } + + public OceanBaseConnectorOptions getOceanBaseConnectorOptions() { + Map options = new HashMap<>(); + if (sinkConfig != null) { + options = sinkConfig.toMap(); + return new OceanBaseConnectorOptions(options); + } + throw new RuntimeException("sinkConfig is null"); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java new file mode 100644 index 0000000..e73054c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/CdcTools.java @@ -0,0 +1,215 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.cdc.db2.Db2DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.mysql.MysqlDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.oracle.OracleDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.postgres.PostgresDatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.sqlserver.SqlServerDatabaseSync; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.utils.MultipleParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** cdc sync tools. */ +public class CdcTools { + private static final List EMPTY_KEYS = + Collections.singletonList(DatabaseSyncConfig.PASSWORD); + private static StreamExecutionEnvironment flinkEnvironmentForTesting; + private static JobClient jobClient; + + public static void main(String[] args) throws Exception { + System.out.println("Input args: " + Arrays.asList(args) + ".\n"); + String operation = args[0].toLowerCase(); + String[] opArgs = Arrays.copyOfRange(args, 1, args.length); + MultipleParameterTool params = MultipleParameterTool.fromArgs(opArgs); + switch (operation) { + // mysql is synchronized as a data source + case DatabaseSyncConfig.MYSQL_SYNC_DATABASE: + createMySQLSyncDatabase(params); + break; + // oracle is synchronized as a data source + case DatabaseSyncConfig.ORACLE_SYNC_DATABASE: + createOracleSyncDatabase(params); + break; + // postgres is synchronized as a data source + case DatabaseSyncConfig.POSTGRES_SYNC_DATABASE: + createPostgresSyncDatabase(params); + break; + // sqlserver is synchronized as a data source + case DatabaseSyncConfig.SQLSERVER_SYNC_DATABASE: + createSqlServerSyncDatabase(params); + break; + // db2 is synchronized as a data source + case DatabaseSyncConfig.DB2_SYNC_DATABASE: + createDb2SyncDatabase(params); + break; + default: + System.out.println("Unknown operation " + operation); + System.exit(1); + } + } + + private static void createMySQLSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.MYSQL_CONF)); + Map mysqlMap = getConfigMap(params, DatabaseSyncConfig.MYSQL_CONF); + Configuration mysqlConfig = Configuration.fromMap(mysqlMap); + DatabaseSync databaseSync = new MysqlDatabaseSync(); + syncDatabase(params, databaseSync, mysqlConfig, SourceConnector.MYSQL); + } + + private static void createOracleSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.ORACLE_CONF)); + Map oracleMap = getConfigMap(params, DatabaseSyncConfig.ORACLE_CONF); + Configuration oracleConfig = Configuration.fromMap(oracleMap); + DatabaseSync databaseSync = new OracleDatabaseSync(); + syncDatabase(params, databaseSync, oracleConfig, SourceConnector.ORACLE); + } + + private static void createPostgresSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.POSTGRES_CONF)); + Map postgresMap = getConfigMap(params, DatabaseSyncConfig.POSTGRES_CONF); + Configuration postgresConfig = Configuration.fromMap(postgresMap); + DatabaseSync databaseSync = new PostgresDatabaseSync(); + syncDatabase(params, databaseSync, postgresConfig, SourceConnector.POSTGRES); + } + + private static void createSqlServerSyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.SQLSERVER_CONF)); + Map postgresMap = getConfigMap(params, DatabaseSyncConfig.SQLSERVER_CONF); + Configuration postgresConfig = Configuration.fromMap(postgresMap); + DatabaseSync databaseSync = new SqlServerDatabaseSync(); + syncDatabase(params, databaseSync, postgresConfig, SourceConnector.SQLSERVER); + } + + private static void createDb2SyncDatabase(MultipleParameterTool params) throws Exception { + Preconditions.checkArgument(params.has(DatabaseSyncConfig.DB2_CONF)); + Map db2Map = getConfigMap(params, DatabaseSyncConfig.DB2_CONF); + Configuration db2Config = Configuration.fromMap(db2Map); + DatabaseSync databaseSync = new Db2DatabaseSync(); + syncDatabase(params, databaseSync, db2Config, SourceConnector.DB2); + } + + private static void syncDatabase( + MultipleParameterTool params, + DatabaseSync databaseSync, + Configuration config, + SourceConnector sourceConnector) + throws Exception { + String jobName = params.get(DatabaseSyncConfig.JOB_NAME); + String database = params.get(DatabaseSyncConfig.DATABASE); + String tablePrefix = params.get(DatabaseSyncConfig.TABLE_PREFIX); + String tableSuffix = params.get(DatabaseSyncConfig.TABLE_SUFFIX); + String includingTables = params.get(DatabaseSyncConfig.INCLUDING_TABLES); + String excludingTables = params.get(DatabaseSyncConfig.EXCLUDING_TABLES); + String multiToOneOrigin = params.get(DatabaseSyncConfig.MULTI_TO_ONE_ORIGIN); + String multiToOneTarget = params.get(DatabaseSyncConfig.MULTI_TO_ONE_TARGET); + boolean createTableOnly = params.has(DatabaseSyncConfig.CREATE_TABLE_ONLY); + boolean ignoreDefaultValue = params.has(DatabaseSyncConfig.IGNORE_DEFAULT_VALUE); + boolean ignoreIncompatible = params.has(DatabaseSyncConfig.IGNORE_INCOMPATIBLE); + + Preconditions.checkArgument(params.has(DatabaseSyncConfig.SINK_CONF)); + Map sinkMap = getConfigMap(params, DatabaseSyncConfig.SINK_CONF); + Configuration sinkConfig = Configuration.fromMap(sinkMap); + + StreamExecutionEnvironment env = + Objects.nonNull(flinkEnvironmentForTesting) + ? flinkEnvironmentForTesting + : StreamExecutionEnvironment.getExecutionEnvironment(); + databaseSync + .setEnv(env) + .setDatabase(database) + .setConfig(config) + .setTablePrefix(tablePrefix) + .setTableSuffix(tableSuffix) + .setIncludingTables(includingTables) + .setExcludingTables(excludingTables) + .setMultiToOneOrigin(multiToOneOrigin) + .setMultiToOneTarget(multiToOneTarget) + .setIgnoreDefaultValue(ignoreDefaultValue) + .setSinkConfig(sinkConfig) + .setCreateTableOnly(createTableOnly) + .create(); + databaseSync.build(); + if (StringUtils.isNullOrWhitespaceOnly(jobName)) { + jobName = + String.format( + "%s-OceanBase Sync Database: %s", + sourceConnector.getConnectorName(), + config.getString( + DatabaseSyncConfig.DATABASE_NAME, DatabaseSyncConfig.DB)); + } + if (Objects.nonNull(flinkEnvironmentForTesting)) { + jobClient = env.executeAsync(); + } else { + env.execute(jobName); + } + } + + @VisibleForTesting + public static JobClient getJobClient() { + return jobClient; + } + + // Only for testing, please do not use it in actual environment + @VisibleForTesting + public static void setStreamExecutionEnvironmentForTesting( + StreamExecutionEnvironment environment) { + flinkEnvironmentForTesting = environment; + } + + @VisibleForTesting + public static Map getConfigMap(MultipleParameterTool params, String key) { + if (!params.has(key)) { + System.out.println( + "Can not find key [" + + key + + "] from args: " + + params.toMap().toString() + + ".\n"); + return null; + } + + Map map = new HashMap<>(); + for (String param : params.getMultiParameter(key)) { + String[] kv = param.split("=", 2); + if (kv.length == 2) { + map.put(kv[0].trim(), kv[1].trim()); + continue; + } else if (kv.length == 1 && EMPTY_KEYS.contains(kv[0])) { + map.put(kv[0].trim(), ""); + continue; + } + + System.out.println("Invalid " + key + " " + param + ".\n"); + return null; + } + return map; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java new file mode 100644 index 0000000..fb6b7e9 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSync.java @@ -0,0 +1,401 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc; // Licensed to the Apache Software Foundation (ASF) + +import com.oceanbase.connector.flink.connection.OceanBaseToolsConnectProvider; +import com.oceanbase.connector.flink.tools.catalog.OceanBaseSchemaFactory; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; +import com.oceanbase.connector.flink.tools.catalog.oceanBaseSinkOperate; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.SQLSyntaxErrorException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.flink.cdc.debezium.utils.JdbcUrlUtils.PROPERTIES_PREFIX; + +public abstract class DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(DatabaseSync.class); + private static final String TABLE_NAME_OPTIONS = "table-name"; + + protected Configuration config; + + protected String database; + + protected TableNameConverter converter; + protected Pattern includingPattern; + protected Pattern excludingPattern; + protected Map multiToOneRulesPattern; + protected Configuration sinkConfig; + protected boolean ignoreDefaultValue; + protected boolean ignoreIncompatible; + + public StreamExecutionEnvironment env; + private boolean createTableOnly = false; + protected String includingTables; + protected String excludingTables; + protected String multiToOneOrigin; + protected String multiToOneTarget; + protected String tablePrefix; + protected String tableSuffix; + protected final Map tableMapping = new HashMap<>(); + public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; + + public abstract void registerDriver() throws SQLException; + + public abstract Connection getConnection() throws SQLException; + + public abstract List getSchemaList() throws Exception; + + public abstract DataStreamSource buildCdcSource(StreamExecutionEnvironment env); + + /** Get the prefix of a specific tableList, for example, mysql is database, oracle is schema. */ + public abstract String getTableListPrefix(); + + protected DatabaseSync() throws SQLException { + registerDriver(); + } + + public void create() { + this.includingPattern = includingTables == null ? null : Pattern.compile(includingTables); + this.excludingPattern = excludingTables == null ? null : Pattern.compile(excludingTables); + this.multiToOneRulesPattern = multiToOneRulesParser(multiToOneOrigin, multiToOneTarget); + this.converter = new TableNameConverter(tablePrefix, tableSuffix, multiToOneRulesPattern); + } + + public void build() throws Exception { + oceanBaseSinkOperate oceanBaseSinkOperate = new oceanBaseSinkOperate(sinkConfig); + OceanBaseToolsConnectProvider oceanBaseConnectionProvider = + new OceanBaseToolsConnectProvider( + oceanBaseSinkOperate.getOceanBaseConnectorOptions()); + List schemaList = getSchemaList(); + Preconditions.checkState( + !schemaList.isEmpty(), + "No tables to be synchronized. Please make sure whether the tables that need to be synchronized exist in the corresponding database or schema."); + + if (!StringUtils.isNullOrWhitespaceOnly(database) + && !oceanBaseConnectionProvider.databaseExists(database)) { + LOG.info("database {} not exist, created", database); + oceanBaseConnectionProvider.createDatabase(database); + } + List syncTables = new ArrayList<>(); + List> oceanbaseTables = new ArrayList<>(); + + Set targetDbSet = new HashSet<>(); + for (SourceSchema schema : schemaList) { + syncTables.add(schema.getTableName()); + String targetDb = database; + // Synchronize multiple databases using the src database name + if (StringUtils.isNullOrWhitespaceOnly(targetDb)) { + targetDb = schema.getDatabaseName(); + targetDbSet.add(targetDb); + } + if (StringUtils.isNullOrWhitespaceOnly(database) + && !oceanBaseConnectionProvider.databaseExists(targetDb)) { + LOG.info("database {} not exist, created", targetDb); + oceanBaseConnectionProvider.createDatabase(targetDb); + } + String oceanbaseTable = converter.convert(schema.getTableName()); + // Calculate the mapping relationship between upstream and downstream tables + tableMapping.put( + schema.getTableIdentifier(), String.format("%s.%s", targetDb, oceanbaseTable)); + tryCreateTableIfAbsent(oceanBaseConnectionProvider, targetDb, oceanbaseTable, schema); + + if (!oceanbaseTables.contains(Tuple2.of(targetDb, oceanbaseTable))) { + oceanbaseTables.add(Tuple2.of(targetDb, oceanbaseTable)); + } + } + if (createTableOnly) { + LOG.info("Create table finished."); + System.exit(0); + } + LOG.info("table mapping: {}", tableMapping); + config.setString(TABLE_NAME_OPTIONS, getSyncTableList(syncTables)); + DataStreamSource streamSource = buildCdcSource(env); + SingleOutputStreamOperator parsedStream = + streamSource.process(buildProcessFunction()); + for (Tuple2 dbTbl : oceanbaseTables) { + OutputTag recordOutputTag = + ParsingProcessFunction.createRecordOutputTag(dbTbl.f1); + DataStream sideOutput = parsedStream.getSideOutput(recordOutputTag); + int sinkParallel = sinkConfig.getInteger(SINK_PARALLELISM, sideOutput.getParallelism()); + String uidName = getUidName(targetDbSet, dbTbl); + sideOutput + .sinkTo(oceanBaseSinkOperate.createGenericOceanBaseSink(dbTbl.f0, dbTbl.f1)) + .setParallelism(sinkParallel) + .name(uidName) + .uid(uidName); + } + } + + /** + * @param targetDbSet The set of target databases. + * @param dbTbl The database-table tuple. + * @return The UID of the DataStream. + */ + public String getUidName(Set targetDbSet, Tuple2 dbTbl) { + String uidName; + // Determine whether to proceed with multi-database synchronization. + // if yes, the UID is composed of `dbname_tablename`, otherwise it is composed of + // `tablename`. + if (targetDbSet.size() > 1) { + uidName = dbTbl.f0 + "_" + dbTbl.f1; + } else { + uidName = dbTbl.f1; + } + + return uidName; + } + + public ParsingProcessFunction buildProcessFunction() { + return new ParsingProcessFunction(converter); + } + + /** Filter table that need to be synchronized. */ + protected boolean isSyncNeeded(String tableName) { + boolean sync = true; + if (includingPattern != null) { + sync = includingPattern.matcher(tableName).matches(); + } + if (excludingPattern != null) { + sync = sync && !excludingPattern.matcher(tableName).matches(); + } + LOG.debug("table {} is synchronized? {}", tableName, sync); + return sync; + } + + protected String getSyncTableList(List syncTables) { + // includingTablePattern and ^excludingPattern + if (includingTables == null) { + includingTables = ".*"; + } + String includingPattern = + String.format("(%s)\\.(%s)", getTableListPrefix(), includingTables); + if (StringUtils.isNullOrWhitespaceOnly(excludingTables)) { + return includingPattern; + } else { + String excludingPattern = + String.format("?!(%s\\.(%s))$", getTableListPrefix(), excludingTables); + return String.format("(%s)(%s)", excludingPattern, includingPattern); + } + } + + /** Filter table that many tables merge to one. */ + protected HashMap multiToOneRulesParser( + String multiToOneOrigin, String multiToOneTarget) { + if (StringUtils.isNullOrWhitespaceOnly(multiToOneOrigin) + || StringUtils.isNullOrWhitespaceOnly(multiToOneTarget)) { + return null; + } + HashMap multiToOneRulesPattern = new HashMap<>(); + String[] origins = multiToOneOrigin.split("\\|"); + String[] targets = multiToOneTarget.split("\\|"); + if (origins.length != targets.length) { + LOG.error( + "param error : multi to one params length are not equal,please check your params."); + System.exit(1); + } + try { + for (int i = 0; i < origins.length; i++) { + multiToOneRulesPattern.put(Pattern.compile(origins[i]), targets[i]); + } + } catch (Exception e) { + LOG.error("param error : Your regular expression is incorrect,please check."); + System.exit(1); + } + return multiToOneRulesPattern; + } + + private void tryCreateTableIfAbsent( + OceanBaseToolsConnectProvider connectionProvider, + String targetDb, + String oceanbaseTable, + SourceSchema schema) { + if (!connectionProvider.tableExists(targetDb, oceanbaseTable)) { + TableSchema oceanbaseSchema = + OceanBaseSchemaFactory.createTableSchema( + database, + oceanbaseTable, + schema.getFields(), + schema.getPrimaryKeys(), + schema.getTableComment()); + try { + connectionProvider.createTable(oceanbaseSchema); + } catch (Exception ex) { + handleTableCreationFailure(ex); + } + } + } + + private void handleTableCreationFailure(Exception ex) { + if (ignoreIncompatible && ex.getCause() instanceof SQLSyntaxErrorException) { + LOG.warn( + "OceanBase schema and source table schema are not compatible. Error: {} ", + ex.getCause().toString()); + } else { + throw new RuntimeException("Failed to create table due to: ", ex); + } + } + + protected Properties getJdbcProperties() { + Properties jdbcProps = new Properties(); + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(PROPERTIES_PREFIX)) { + jdbcProps.put(key.substring(PROPERTIES_PREFIX.length()), value); + } + } + return jdbcProps; + } + + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + jdbcProperties.forEach( + (key, value) -> jdbcUrlBuilder.append("&").append(key).append("=").append(value)); + return jdbcUrlBuilder.toString(); + } + + public DatabaseSync setEnv(StreamExecutionEnvironment env) { + this.env = env; + return this; + } + + public DatabaseSync setConfig(Configuration config) { + this.config = config; + return this; + } + + public DatabaseSync setDatabase(String database) { + this.database = database; + return this; + } + + public DatabaseSync setIncludingTables(String includingTables) { + this.includingTables = includingTables; + return this; + } + + public DatabaseSync setExcludingTables(String excludingTables) { + this.excludingTables = excludingTables; + return this; + } + + public DatabaseSync setMultiToOneOrigin(String multiToOneOrigin) { + this.multiToOneOrigin = multiToOneOrigin; + return this; + } + + public DatabaseSync setMultiToOneTarget(String multiToOneTarget) { + this.multiToOneTarget = multiToOneTarget; + return this; + } + + public DatabaseSync setSinkConfig(Configuration sinkConfig) { + this.sinkConfig = sinkConfig; + return this; + } + + public DatabaseSync setIgnoreDefaultValue(boolean ignoreDefaultValue) { + this.ignoreDefaultValue = ignoreDefaultValue; + return this; + } + + public DatabaseSync setCreateTableOnly(boolean createTableOnly) { + this.createTableOnly = createTableOnly; + return this; + } + + public DatabaseSync setTablePrefix(String tablePrefix) { + this.tablePrefix = tablePrefix; + return this; + } + + public DatabaseSync setTableSuffix(String tableSuffix) { + this.tableSuffix = tableSuffix; + return this; + } + + public static class TableNameConverter implements Serializable { + private static final long serialVersionUID = 1L; + private final String prefix; + private final String suffix; + private Map multiToOneRulesPattern; + + TableNameConverter() { + this("", ""); + } + + TableNameConverter(String prefix, String suffix) { + this.prefix = prefix == null ? "" : prefix; + this.suffix = suffix == null ? "" : suffix; + } + + TableNameConverter( + String prefix, String suffix, Map multiToOneRulesPattern) { + this.prefix = prefix == null ? "" : prefix; + this.suffix = suffix == null ? "" : suffix; + this.multiToOneRulesPattern = multiToOneRulesPattern; + } + + public String convert(String tableName) { + if (multiToOneRulesPattern == null) { + return prefix + tableName + suffix; + } + + String target = null; + + for (Map.Entry patternStringEntry : + multiToOneRulesPattern.entrySet()) { + if (patternStringEntry.getKey().matcher(tableName).matches()) { + target = patternStringEntry.getValue(); + } + } + /** + * If multiToOneRulesPattern is not null and target is not assigned, then the + * synchronization task contains both multi to one and one to one , prefixes and + * suffixes are added to common one-to-one mapping tables + */ + if (target == null) { + return prefix + tableName + suffix; + } + return target; + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java new file mode 100644 index 0000000..ec6b72d --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/DatabaseSyncConfig.java @@ -0,0 +1,91 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +public class DatabaseSyncConfig { + + public static final String MYSQL_SYNC_DATABASE = "mysql-sync-database"; + public static final String ORACLE_SYNC_DATABASE = "oracle-sync-database"; + public static final String POSTGRES_SYNC_DATABASE = "postgres-sync-database"; + public static final String SQLSERVER_SYNC_DATABASE = "sqlserver-sync-database"; + public static final String DB2_SYNC_DATABASE = "db2-sync-database"; + + public static final String MYSQL_CONF = "mysql-conf"; + public static final String ORACLE_CONF = "oracle-conf"; + public static final String POSTGRES_CONF = "postgres-conf"; + public static final String SQLSERVER_CONF = "sqlserver-conf"; + public static final String DB2_CONF = "db2-conf"; + + ///////////// source-conf //////// + public static final String DATABASE_NAME = "database-name"; + public static final String DB = "db"; + public static final String USER = "user"; + public static final String PASSWORD = "password"; + public static final String TABLE_NAME = "TABLE_NAME"; + public static final String REMARKS = "REMARKS"; + + ////////// cdc-conf ////////// + // config options of {@link + // org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE} + public static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + public static final String SCAN_STARTUP_MODE_VALUE_EARLIEST_OFFSET = "earliest-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET = "specific-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET = "latest-offset"; + public static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp"; + public static final String DECIMAL_HANDLING_MODE = "decimal.handling.mode"; + + ////////// sink-conf ///////////// + public static final String SINK_CONF = "sink-conf"; + public static final String JOB_NAME = "job-name"; + public static final String DATABASE = "database"; + public static final String TABLE_PREFIX = "table-prefix"; + public static final String TABLE_SUFFIX = "table-suffix"; + public static final String INCLUDING_TABLES = "including-tables"; + public static final String EXCLUDING_TABLES = "excluding-tables"; + public static final String MULTI_TO_ONE_ORIGIN = "multi-to-one-origin"; + public static final String MULTI_TO_ONE_TARGET = "multi-to-one-target"; + public static final String CREATE_TABLE_ONLY = "create-table-only"; + public static final String IGNORE_DEFAULT_VALUE = "ignore-default-value"; + public static final String IGNORE_INCOMPATIBLE = "ignore-incompatible"; + ////////// oceanbase-table-conf ////////// + public static final String TABLE_CONF = "table-conf"; + + ////////// date-converter-conf ////////// + public static final String CONVERTERS = "converters"; + public static final String DATE = "date"; + public static final String DATE_TYPE = "date.type"; + public static final String DATE_FORMAT_DATE = "date.format.date"; + public static final String DATE_FORMAT_DATETIME = "date.format.datetime"; + public static final String DATE_FORMAT_TIMESTAMP = "date.format.timestamp"; + public static final String DATE_FORMAT_TIMESTAMP_ZONE = "date.format.timestamp.zone"; + public static final String YEAR_MONTH_DAY_FORMAT = "yyyy-MM-dd"; + public static final String DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss"; + public static final String DATETIME_MICRO_FORMAT = "yyyy-MM-dd HH:mm:ss.SSSSSS"; + public static final String TIME_ZONE_SHANGHAI = "Asia/Shanghai"; + public static final String TIME_ZONE_UTC_8 = "UTC+8"; + public static final String FORMAT_DATE = "format.date"; + public static final String FORMAT_TIME = "format.time"; + public static final String FORMAT_DATETIME = "format.datetime"; + public static final String FORMAT_TIMESTAMP = "format.timestamp"; + public static final String FORMAT_TIMESTAMP_ZONE = "format.timestamp.zone"; + public static final String UPPERCASE_DATE = "DATE"; + public static final String TIME = "TIME"; + public static final String DATETIME = "DATETIME"; + public static final String TIMESTAMP = "TIMESTAMP"; + public static final String SMALLDATETIME = "SMALLDATETIME"; + public static final String DATETIME2 = "DATETIME2"; +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java new file mode 100644 index 0000000..4e26cff --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/JdbcSourceSchema.java @@ -0,0 +1,105 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; + +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; + +/** + * JdbcSourceSchema is a subclass of SourceSchema, used to build metadata about jdbc-related + * databases. + */ +public abstract class JdbcSourceSchema extends SourceSchema { + private static final Logger LOG = LoggerFactory.getLogger(JdbcSourceSchema.class); + + public JdbcSourceSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(databaseName, schemaName, tableName, tableComment); + fields = getColumnInfo(metaData, databaseName, schemaName, tableName); + primaryKeys = getPrimaryKeys(metaData, databaseName, schemaName, tableName); + } + + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + LinkedHashMap fields = new LinkedHashMap<>(); + LOG.debug("Starting to get column info for table: {}", tableName); + try (ResultSet rs = metaData.getColumns(databaseName, schemaName, tableName, null)) { + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + String comment = rs.getString("REMARKS"); + String fieldType = rs.getString("TYPE_NAME"); + Integer precision = rs.getInt("COLUMN_SIZE"); + Boolean isNullable = rs.getBoolean("IS_NULLABLE"); + String columnDefault = rs.getString("COLUMN_DEF"); // 默认值 + if (rs.wasNull()) { + precision = null; + } + Integer scale = rs.getInt("DECIMAL_DIGITS"); + if (rs.wasNull()) { + scale = null; + } + String oceanbaseTypeStr = null; + try { + oceanbaseTypeStr = convertToOceanBaseType(fieldType, precision, scale); + } catch (UnsupportedOperationException e) { + throw new UnsupportedOperationException(e + " in table: " + tableName); + } + fields.put( + fieldName, + new FieldSchema( + fieldName, oceanbaseTypeStr, columnDefault, comment, isNullable)); + } + } + Preconditions.checkArgument(!fields.isEmpty(), "The column info of {} is empty", tableName); + LOG.debug("Successfully retrieved column info for table: {}", tableName); + return fields; + } + + public List getPrimaryKeys( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + List primaryKeys = new ArrayList<>(); + try (ResultSet rs = metaData.getPrimaryKeys(databaseName, schemaName, tableName)) { + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } + + return primaryKeys; + } + + public abstract String convertToOceanBaseType( + String fieldType, Integer precision, Integer scale); +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java new file mode 100644 index 0000000..b64f041 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/ParsingProcessFunction.java @@ -0,0 +1,70 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.HashMap; +import java.util.Map; + +public class ParsingProcessFunction extends ProcessFunction { + protected ObjectMapper objectMapper = new ObjectMapper(); + private transient Map> recordOutputTags; + private DatabaseSync.TableNameConverter converter; + + public ParsingProcessFunction(DatabaseSync.TableNameConverter converter) { + this.converter = converter; + } + + @Override + public void open(Configuration parameters) throws Exception { + recordOutputTags = new HashMap<>(); + } + + @Override + public void processElement( + String record, ProcessFunction.Context context, Collector collector) + throws Exception { + String tableName = getRecordTableName(record); + String oceanbaseName = converter.convert(tableName); + context.output(getRecordOutputTag(oceanbaseName), record); + } + + protected String getRecordTableName(String record) throws Exception { + JsonNode recordRoot = objectMapper.readValue(record, JsonNode.class); + return extractJsonNode(recordRoot.get("source"), "table"); + } + + protected String extractJsonNode(JsonNode record, String key) { + return record != null && record.get(key) != null ? record.get(key).asText() : null; + } + + private OutputTag getRecordOutputTag(String tableName) { + return recordOutputTags.computeIfAbsent( + tableName, ParsingProcessFunction::createRecordOutputTag); + } + + public static OutputTag createRecordOutputTag(String tableName) { + return new OutputTag("record-" + tableName) {}; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java new file mode 100644 index 0000000..8d15b81 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceConnector.java @@ -0,0 +1,36 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +public enum SourceConnector { + MYSQL("mysql"), + ORACLE("oracle"), + POSTGRES("postgres"), + SQLSERVER("sqlserver"), + MONGODB("mongodb"), + DB2("db2"); + + public final String connectorName; + + SourceConnector(String connectorName) { + this.connectorName = connectorName; + } + + public String getConnectorName() { + return connectorName; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java new file mode 100644 index 0000000..29debd2 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/SourceSchema.java @@ -0,0 +1,96 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; + +import org.apache.flink.util.StringUtils; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.StringJoiner; + +public abstract class SourceSchema { + protected final String databaseName; + protected final String schemaName; + protected final String tableName; + protected final String tableComment; + protected LinkedHashMap fields; + public List primaryKeys; + + public SourceSchema( + String databaseName, String schemaName, String tableName, String tableComment) + throws Exception { + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + this.tableComment = tableComment; + } + + public abstract String convertToOceanBaseType( + String fieldType, Integer precision, Integer scale); + + public String getTableIdentifier() { + return getString(databaseName, schemaName, tableName); + } + + public static String getString(String databaseName, String schemaName, String tableName) { + StringJoiner identifier = new StringJoiner("."); + if (!StringUtils.isNullOrWhitespaceOnly(databaseName)) { + identifier.add(databaseName); + } + if (!StringUtils.isNullOrWhitespaceOnly(schemaName)) { + identifier.add(schemaName); + } + + if (!StringUtils.isNullOrWhitespaceOnly(tableName)) { + identifier.add(tableName); + } + + return identifier.toString(); + } + + @Deprecated + public TableSchema convertTableSchema(Map tableProps) { + TableSchema tableSchema = new TableSchema(); + tableSchema.setFields(this.fields); + tableSchema.setTableComment(this.tableComment); + return tableSchema; + } + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public Map getFields() { + return fields; + } + + public List getPrimaryKeys() { + return primaryKeys; + } + + public String getTableComment() { + return tableComment; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java new file mode 100644 index 0000000..c7dc5bf --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DatabaseSync.java @@ -0,0 +1,241 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.db2.Db2Source; +import org.apache.flink.cdc.connectors.db2.source.Db2SourceBuilder; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SERVER_TIME_ZONE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class Db2DatabaseSync extends DatabaseSync { + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(50000) + .withDescription("Integer port number of the DB2 database server."); + private static final Logger LOG = LoggerFactory.getLogger(Db2DatabaseSync.class); + + private static final String JDBC_URL = "jdbc:db2://%s:%d/%s"; + + public Db2DatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.ibm.db2.jcc.DB2Driver"); + LOG.info(" Loaded the JDBC driver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class com.ibm.db2.jcc.DB2Driver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(JdbcSourceOptions.HOSTNAME), + config.get(PORT), + config.get(JdbcSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty("user", config.get(JdbcSourceOptions.USERNAME)); + pro.setProperty("password", config.get(JdbcSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(null, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString("TABLE_NAME"); + String tableComment = tables.getString("REMARKS"); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new Db2Schema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in DB2 is required"); + Preconditions.checkNotNull(schemaName, "schema-name in DB2 is required"); + + String tableName = config.get(JdbcSourceOptions.TABLE_NAME); + String hostname = config.get(JdbcSourceOptions.HOSTNAME); + Integer port = config.get(PORT); + String username = config.get(JdbcSourceOptions.USERNAME); + String password = config.get(JdbcSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(SourceOptions.SCAN_STARTUP_MODE); + if ("initial".equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if ("latest-offset".equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(Db2DateConverter.DEFAULT_PROPS); + debeziumProperties.put("decimal.handling.mode", "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, true)) { + JdbcIncrementalSource db2IncrementalSource = + Db2SourceBuilder.Db2IncrementalSource.builder() + .hostname(hostname) + .port(port) + .databaseList(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .deserializer(schema) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .serverTimeZone(config.get(SERVER_TIME_ZONE)) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + db2IncrementalSource, WatermarkStrategy.noWatermarks(), "Db2IncrementalSource"); + + } else { + DebeziumSourceFunction db2Source = + Db2Source.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(db2Source, "Db2 Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(JdbcSourceOptions.SCHEMA_NAME); + } + + @Override + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + boolean firstParam = true; + for (Map.Entry entry : jdbcProperties.entrySet()) { + Object key = entry.getKey(); + Object value = entry.getValue(); + if (firstParam) { + jdbcUrlBuilder.append(":").append(key).append("=").append(value).append(";"); + firstParam = false; + } else { + jdbcUrlBuilder.append(key).append("=").append(value).append(";"); + } + } + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java new file mode 100644 index 0000000..7f47bcd --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2DateConverter.java @@ -0,0 +1,132 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class Db2DateConverter implements CustomConverter { + private static final Logger LOGGER = LoggerFactory.getLogger(Db2DateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + private final DateTimeFormatter timeFormatter = DateTimeFormatter.ISO_TIME; + + protected static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.db2.Db2DateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps(props, "format.date", p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + "format.timestamp", + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.isEmpty()) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + LOGGER.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } else if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof Time) { + return timeFormatter.format(((Time) input).toLocalTime()); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } else if (input instanceof Instant) { + LocalDateTime ldt = LocalDateTime.ofInstant(((Instant) input), ZoneOffset.UTC); + return timestampFormatter.format(ldt); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java new file mode 100644 index 0000000..3e416f3 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Schema.java @@ -0,0 +1,47 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; +import java.sql.SQLException; +import java.util.LinkedHashMap; + +public class Db2Schema extends JdbcSourceSchema { + public Db2Schema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return Db2Type.toOceanBaseType(fieldType, precision, scale); + } + + @Override + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + return super.getColumnInfo(metaData, null, schemaName, tableName); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java new file mode 100644 index 0000000..8e9fae4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/db2/Db2Type.java @@ -0,0 +1,93 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.db2; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class Db2Type { + private static final String BOOLEAN = "BOOLEAN"; + private static final String SMALLINT = "SMALLINT"; + private static final String INTEGER = "INTEGER"; + private static final String INT = "INT"; + private static final String BIGINT = "BIGINT"; + private static final String REAL = "REAL"; + private static final String DECFLOAT = "DECFLOAT"; + private static final String DOUBLE = "DOUBLE"; + private static final String DECIMAL = "DECIMAL"; + private static final String NUMERIC = "NUMERIC"; + private static final String DATE = "DATE"; + private static final String TIME = "TIME"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String CHARACTER = "CHARACTER"; + private static final String CHAR = "CHAR"; + private static final String LONG_VARCHAR = "LONG VARCHAR"; + private static final String VARCHAR = "VARCHAR"; + private static final String XML = "XML"; + private static final String VARGRAPHIC = "VARGRAPHIC"; + + public static String toOceanBaseType(String db2Type, Integer precision, Integer scale) { + db2Type = db2Type.toUpperCase(); + switch (db2Type) { + case BOOLEAN: + return OceanBaseType.BOOLEAN; + case SMALLINT: + return OceanBaseType.SMALLINT; + case INTEGER: + case INT: + return OceanBaseType.INT; + case BIGINT: + return OceanBaseType.BIGINT; + case REAL: + return OceanBaseType.FLOAT; + case DOUBLE: + return OceanBaseType.DOUBLE; + case DATE: + return OceanBaseType.DATE; + case DECFLOAT: + case DECIMAL: + case NUMERIC: + if (precision != null && precision > 0 && precision <= 38) { + if (scale != null && scale >= 0) { + return String.format("%s(%s,%s)", OceanBaseType.DECIMAL, precision, scale); + } + return String.format("%s(%s,%s)", OceanBaseType.DECIMAL, precision, 0); + } else { + return OceanBaseType.VARCHAR; + } + case CHARACTER: + case CHAR: + case VARCHAR: + case LONG_VARCHAR: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case TIMESTAMP: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case TIME: + case VARGRAPHIC: + // Currently, the Flink CDC connector does not support the XML data type from DB2. + // Case XML: + return OceanBaseType.VARCHAR; + default: + throw new UnsupportedOperationException("Unsupported DB2 Type: " + db2Type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java new file mode 100644 index 0000000..42ca5b1 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/deserialize/OceanBaseJsonDebeziumDeserializationSchema.java @@ -0,0 +1,196 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.deserialize; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.ConnectSchema; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Field; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Schema; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.Struct; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.source.SourceRecord; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.util.Collector; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.Map; + +/** Currently just use for synchronous mysql non-default. */ +public class OceanBaseJsonDebeziumDeserializationSchema + implements DebeziumDeserializationSchema { + + private static final JsonNodeFactory JSON_NODE_FACTORY = + JsonNodeFactory.withExactBigDecimals(true); + private final ObjectMapper objectMapper; + + public OceanBaseJsonDebeziumDeserializationSchema() { + objectMapper = new ObjectMapper(); + } + + @Override + public void deserialize(SourceRecord sourceRecord, Collector collector) + throws Exception { + Schema schema = sourceRecord.valueSchema(); + Object value = sourceRecord.value(); + JsonNode jsonValue = convertToJson(schema, value); + byte[] bytes = objectMapper.writeValueAsString(jsonValue).getBytes(StandardCharsets.UTF_8); + collector.collect(new String(bytes)); + } + + private JsonNode convertToJson(Schema schema, Object value) throws RuntimeException { + if (value == null) { + // Any schema is valid and we don't have a default, so treat this as an optional schema + if (schema == null) { + return null; + } + if (schema.isOptional()) { + return JSON_NODE_FACTORY.nullNode(); + } + throw new RuntimeException( + "Conversion error: null value for field that is required and has no default value"); + } + + try { + final Schema.Type schemaType; + if (schema == null) { + schemaType = ConnectSchema.schemaType(value.getClass()); + if (schemaType == null) { + throw new RuntimeException( + "Java class " + + value.getClass() + + " does not have corresponding schema type."); + } + } else { + schemaType = schema.type(); + } + switch (schemaType) { + case INT8: + return JSON_NODE_FACTORY.numberNode((Byte) value); + case INT16: + return JSON_NODE_FACTORY.numberNode((Short) value); + case INT32: + return JSON_NODE_FACTORY.numberNode((Integer) value); + case INT64: + return JSON_NODE_FACTORY.numberNode((Long) value); + case FLOAT32: + return JSON_NODE_FACTORY.numberNode((Float) value); + case FLOAT64: + return JSON_NODE_FACTORY.numberNode((Double) value); + case BOOLEAN: + return JSON_NODE_FACTORY.booleanNode((Boolean) value); + case STRING: + CharSequence charSeq = (CharSequence) value; + return JSON_NODE_FACTORY.textNode(charSeq.toString()); + case BYTES: + if (value instanceof byte[]) { + return JSON_NODE_FACTORY.binaryNode((byte[]) value); + } else if (value instanceof ByteBuffer) { + return JSON_NODE_FACTORY.binaryNode(((ByteBuffer) value).array()); + } else if (value instanceof BigDecimal) { + return JSON_NODE_FACTORY.numberNode((BigDecimal) value); + } else { + throw new RuntimeException( + "Invalid type for bytes type: " + value.getClass()); + } + case ARRAY: + { + Collection> collection = (Collection>) value; + ArrayNode list = JSON_NODE_FACTORY.arrayNode(); + for (Object elem : collection) { + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode fieldValue = convertToJson(valueSchema, elem); + list.add(fieldValue); + } + return list; + } + case MAP: + { + Map, ?> map = (Map, ?>) value; + // If true, using string keys and JSON object; if false, using non-string + // keys and Array-encoding + boolean objectMode; + if (schema == null) { + objectMode = true; + for (Map.Entry, ?> entry : map.entrySet()) { + if (!(entry.getKey() instanceof String)) { + objectMode = false; + break; + } + } + } else { + objectMode = schema.keySchema().type() == Schema.Type.STRING; + } + ObjectNode obj = null; + ArrayNode list = null; + if (objectMode) { + obj = JSON_NODE_FACTORY.objectNode(); + } else { + list = JSON_NODE_FACTORY.arrayNode(); + } + for (Map.Entry, ?> entry : map.entrySet()) { + Schema keySchema = schema == null ? null : schema.keySchema(); + Schema valueSchema = schema == null ? null : schema.valueSchema(); + JsonNode mapKey = convertToJson(keySchema, entry.getKey()); + JsonNode mapValue = convertToJson(valueSchema, entry.getValue()); + + if (objectMode) { + obj.set(mapKey.asText(), mapValue); + } else { + list.add(JSON_NODE_FACTORY.arrayNode().add(mapKey).add(mapValue)); + } + } + return objectMode ? obj : list; + } + case STRUCT: + { + Struct struct = (Struct) value; + if (!struct.schema().equals(schema)) { + throw new RuntimeException("Mismatching schema."); + } + ObjectNode obj = JSON_NODE_FACTORY.objectNode(); + for (Field field : schema.fields()) { + obj.set( + field.name(), + convertToJson( + field.schema(), + struct.getWithoutDefault(field.name()))); + } + return obj; + } + } + throw new RuntimeException("Couldn't convert " + value + " to JSON."); + } catch (ClassCastException e) { + String schemaTypeStr = (schema != null) ? schema.type().toString() : "unknown schema"; + throw new RuntimeException( + "Invalid type for " + schemaTypeStr + ": " + value.getClass()); + } + } + + @Override + public TypeInformation getProducedType() { + return BasicTypeInfo.STRING_TYPE_INFO; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java new file mode 100644 index 0000000..2f521f4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/DateToStringConverter.java @@ -0,0 +1,173 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class DateToStringConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(DateToStringConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timeFormatter = DateTimeFormatter.ISO_TIME; + private DateTimeFormatter datetimeFormatter = DateTimeFormatter.ISO_DATE_TIME; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + private ZoneId timestampZoneId = ZoneId.systemDefault(); + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.mysql.DateToStringConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATETIME, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP_ZONE, DatabaseSyncConfig.TIME_ZONE_UTC_8); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIME, + p -> timeFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_DATETIME, + p -> datetimeFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP_ZONE, + z -> timestampZoneId = ZoneId.of(z)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.DATETIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDateTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof Duration) { + Duration duration = (Duration) input; + long seconds = duration.getSeconds(); + int nano = duration.getNano(); + LocalTime time = LocalTime.ofSecondOfDay(seconds).withNano(nano); + return timeFormatter.format(time); + } + return null; + } + + private String convertDateTime(Object input) { + if (input instanceof LocalDateTime) { + return datetimeFormatter.format((LocalDateTime) input); + } else if (input instanceof Timestamp) { + return datetimeFormatter.format(((Timestamp) input).toLocalDateTime()); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof ZonedDateTime) { + // mysql timestamp will be converted to UTC storage, + // and the zonedDatetime here is UTC time + ZonedDateTime zonedDateTime = (ZonedDateTime) input; + LocalDateTime localDateTime = + zonedDateTime.withZoneSameInstant(timestampZoneId).toLocalDateTime(); + return timestampFormatter.format(localDateTime); + } else if (input instanceof Timestamp) { + return timestampFormatter.format( + ((Timestamp) input).toInstant().atZone(timestampZoneId).toLocalDateTime()); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java new file mode 100644 index 0000000..2b89ff4 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlDatabaseSync.java @@ -0,0 +1,264 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.mysql.source.MySqlSource; +import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceBuilder; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetBuilder; +import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.apache.flink.cdc.debezium.utils.JdbcUrlUtils.PROPERTIES_PREFIX; + +public class MysqlDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(MysqlDatabaseSync.class); + private static final String JDBC_URL = "jdbc:mysql://%s:%d?useInformationSchema=true"; + + public MysqlDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.mysql.cj.jdbc.Driver"); + } catch (ClassNotFoundException ex) { + LOG.warn( + "can not found class com.mysql.cj.jdbc.Driver, use class com.mysql.jdbc.Driver"); + try { + Class.forName("com.mysql.jdbc.Driver"); + } catch (Exception e) { + throw new SQLException( + "No suitable driver found, can not found class com.mysql.cj.jdbc.Driver and com.mysql.jdbc.Driver"); + } + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(MySqlSourceOptions.HOSTNAME), + config.get(MySqlSourceOptions.PORT)); + return DriverManager.getConnection( + jdbcUrl, + config.get(MySqlSourceOptions.USERNAME), + config.get(MySqlSourceOptions.PASSWORD)); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(MySqlSourceOptions.DATABASE_NAME); + List schemaList = new ArrayList<>(); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet catalogs = metaData.getCatalogs()) { + while (catalogs.next()) { + String tableCatalog = catalogs.getString("TABLE_CAT"); + if (tableCatalog.matches(databaseName)) { + try (ResultSet tables = + metaData.getTables( + tableCatalog, null, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new MysqlSchema( + metaData, tableCatalog, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + MySqlSourceBuilder sourceBuilder = MySqlSource.builder(); + + String databaseName = config.get(MySqlSourceOptions.DATABASE_NAME); + Preconditions.checkNotNull(databaseName, "database-name in mysql is required"); + String tableName = config.get(MySqlSourceOptions.TABLE_NAME); + sourceBuilder + .hostname(config.get(MySqlSourceOptions.HOSTNAME)) + .port(config.get(MySqlSourceOptions.PORT)) + .username(config.get(MySqlSourceOptions.USERNAME)) + .password(config.get(MySqlSourceOptions.PASSWORD)) + .databaseList(databaseName) + .tableList(tableName); + + config.getOptional(MySqlSourceOptions.SERVER_ID).ifPresent(sourceBuilder::serverId); + config.getOptional(MySqlSourceOptions.SERVER_TIME_ZONE) + .ifPresent(sourceBuilder::serverTimeZone); + config.getOptional(MySqlSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE) + .ifPresent(sourceBuilder::fetchSize); + config.getOptional(MySqlSourceOptions.CONNECT_TIMEOUT) + .ifPresent(sourceBuilder::connectTimeout); + config.getOptional(MySqlSourceOptions.CONNECT_MAX_RETRIES) + .ifPresent(sourceBuilder::connectMaxRetries); + config.getOptional(MySqlSourceOptions.CONNECTION_POOL_SIZE) + .ifPresent(sourceBuilder::connectionPoolSize); + config.getOptional(MySqlSourceOptions.HEARTBEAT_INTERVAL) + .ifPresent(sourceBuilder::heartbeatInterval); + config.getOptional(MySqlSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED) + .ifPresent(sourceBuilder::scanNewlyAddedTableEnabled); + config.getOptional(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE) + .ifPresent(sourceBuilder::splitSize); + config.getOptional(MySqlSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED) + .ifPresent(sourceBuilder::closeIdleReaders); + + setChunkColumns(sourceBuilder); + String startupMode = config.get(MySqlSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + sourceBuilder.startupOptions(StartupOptions.initial()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_EARLIEST_OFFSET.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions(StartupOptions.earliest()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions(StartupOptions.latest()); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET.equalsIgnoreCase( + startupMode)) { + BinlogOffsetBuilder offsetBuilder = BinlogOffset.builder(); + String file = config.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_FILE); + Long pos = config.get(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_POS); + if (file != null && pos != null) { + offsetBuilder.setBinlogFilePosition(file, pos); + } + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_GTID_SET) + .ifPresent(offsetBuilder::setGtidSet); + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_EVENTS) + .ifPresent(offsetBuilder::setSkipEvents); + config.getOptional(MySqlSourceOptions.SCAN_STARTUP_SPECIFIC_OFFSET_SKIP_ROWS) + .ifPresent(offsetBuilder::setSkipRows); + sourceBuilder.startupOptions(StartupOptions.specificOffset(offsetBuilder.build())); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_TIMESTAMP.equalsIgnoreCase( + startupMode)) { + sourceBuilder.startupOptions( + StartupOptions.timestamp( + config.get(MySqlSourceOptions.SCAN_STARTUP_TIMESTAMP_MILLIS))); + } + + Properties jdbcProperties = new Properties(); + Properties debeziumProperties = new Properties(); + // date to string + debeziumProperties.putAll(DateToStringConverter.DEFAULT_PROPS); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(PROPERTIES_PREFIX)) { + jdbcProperties.put(key.substring(PROPERTIES_PREFIX.length()), value); + } else if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + sourceBuilder.jdbcProperties(jdbcProperties); + sourceBuilder.debeziumProperties(debeziumProperties); + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + customConverterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric"); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + MySqlSource mySqlSource = + sourceBuilder.deserializer(schema).includeSchemaChanges(true).build(); + + return env.fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source"); + } + + @Override + public String getTableListPrefix() { + return config.get(MySqlSourceOptions.DATABASE_NAME); + } + + /** + * set chunkkeyColumn,eg: db.table1:column1,db.table2:column2. + * + * @param sourceBuilder + */ + private void setChunkColumns(MySqlSourceBuilder sourceBuilder) { + Map chunkColumnMap = getChunkColumnMap(); + for (Map.Entry entry : chunkColumnMap.entrySet()) { + sourceBuilder.chunkKeyColumn(entry.getKey(), entry.getValue()); + } + } + + private Map getChunkColumnMap() { + Map chunkMap = new HashMap<>(); + String chunkColumn = + config.getString(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN); + if (!StringUtils.isNullOrWhitespaceOnly(chunkColumn)) { + final Pattern chunkPattern = Pattern.compile("(\\S+)\\.(\\S+):(\\S+)"); + String[] tblColumns = chunkColumn.split(","); + for (String tblCol : tblColumns) { + Matcher matcher = chunkPattern.matcher(tblCol); + if (matcher.find()) { + String db = matcher.group(1); + String table = matcher.group(2); + String col = matcher.group(3); + chunkMap.put(new ObjectPath(db, table), col); + } + } + } + return chunkMap; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java new file mode 100644 index 0000000..0941b8c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlSchema.java @@ -0,0 +1,34 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.mysql; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class MysqlSchema extends JdbcSourceSchema { + + public MysqlSchema( + DatabaseMetaData metaData, String databaseName, String tableName, String tableComment) + throws Exception { + super(metaData, databaseName, null, tableName, tableComment); + } + + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return MysqlType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java new file mode 100644 index 0000000..8f23ee7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/mysql/MysqlType.java @@ -0,0 +1,176 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.mysql; + +public class MysqlType { + private static final String BIT = "BIT"; + private static final String BOOLEAN = "BOOLEAN"; + private static final String BOOL = "BOOL"; + private static final String TINYINT = "TINYINT"; + private static final String TINYINT_UNSIGNED = "TINYINT UNSIGNED"; + private static final String TINYINT_UNSIGNED_ZEROFILL = "TINYINT UNSIGNED ZEROFILL"; + private static final String SMALLINT = "SMALLINT"; + private static final String SMALLINT_UNSIGNED = "SMALLINT UNSIGNED"; + private static final String SMALLINT_UNSIGNED_ZEROFILL = "SMALLINT UNSIGNED ZEROFILL"; + private static final String MEDIUMINT = "MEDIUMINT"; + private static final String MEDIUMINT_UNSIGNED = "MEDIUMINT UNSIGNED"; + private static final String MEDIUMINT_UNSIGNED_ZEROFILL = "MEDIUMINT UNSIGNED ZEROFILL"; + private static final String INT = "INT"; + private static final String INT_UNSIGNED = "INT UNSIGNED"; + private static final String INT_UNSIGNED_ZEROFILL = "INT UNSIGNED ZEROFILL"; + private static final String INTEGER = "INTEGER"; + private static final String INTEGER_UNSIGNED = "INTEGER UNSIGNED"; + private static final String INTEGER_UNSIGNED_ZEROFILL = "INTEGER UNSIGNED ZEROFILL"; + private static final String BIGINT = "BIGINT"; + private static final String SERIAL = "SERIAL"; + private static final String BIGINT_UNSIGNED = "BIGINT UNSIGNED"; + private static final String BIGINT_UNSIGNED_ZEROFILL = "BIGINT UNSIGNED ZEROFILL"; + private static final String REAL = "REAL"; + private static final String REAL_UNSIGNED = "REAL UNSIGNED"; + private static final String REAL_UNSIGNED_ZEROFILL = "REAL UNSIGNED ZEROFILL"; + private static final String FLOAT = "FLOAT"; + private static final String FLOAT_UNSIGNED = "FLOAT UNSIGNED"; + private static final String FLOAT_UNSIGNED_ZEROFILL = "FLOAT UNSIGNED ZEROFILL"; + private static final String DOUBLE = "DOUBLE"; + private static final String DOUBLE_UNSIGNED = "DOUBLE UNSIGNED"; + private static final String DOUBLE_UNSIGNED_ZEROFILL = "DOUBLE UNSIGNED ZEROFILL"; + private static final String DOUBLE_PRECISION = "DOUBLE PRECISION"; + private static final String DOUBLE_PRECISION_UNSIGNED = "DOUBLE PRECISION UNSIGNED"; + private static final String DOUBLE_PRECISION_UNSIGNED_ZEROFILL = + "DOUBLE PRECISION UNSIGNED ZEROFILL"; + private static final String NUMERIC = "NUMERIC"; + private static final String NUMERIC_UNSIGNED = "NUMERIC UNSIGNED"; + private static final String NUMERIC_UNSIGNED_ZEROFILL = "NUMERIC UNSIGNED ZEROFILL"; + private static final String FIXED = "FIXED"; + private static final String FIXED_UNSIGNED = "FIXED UNSIGNED"; + private static final String FIXED_UNSIGNED_ZEROFILL = "FIXED UNSIGNED ZEROFILL"; + private static final String DECIMAL = "DECIMAL"; + private static final String DECIMAL_UNSIGNED = "DECIMAL UNSIGNED"; + private static final String DECIMAL_UNSIGNED_ZEROFILL = "DECIMAL UNSIGNED ZEROFILL"; + private static final String CHAR = "CHAR"; + private static final String VARCHAR = "VARCHAR"; + private static final String TINYTEXT = "TINYTEXT"; + private static final String MEDIUMTEXT = "MEDIUMTEXT"; + private static final String TEXT = "TEXT"; + private static final String LONGTEXT = "LONGTEXT"; + private static final String DATE = "DATE"; + private static final String TIME = "TIME"; + private static final String DATETIME = "DATETIME"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String YEAR = "YEAR"; + private static final String BINARY = "BINARY"; + private static final String VARBINARY = "VARBINARY"; + private static final String TINYBLOB = "TINYBLOB"; + private static final String MEDIUMBLOB = "MEDIUMBLOB"; + private static final String BLOB = "BLOB"; + private static final String LONGBLOB = "LONGBLOB"; + private static final String JSON = "JSON"; + private static final String ENUM = "ENUM"; + private static final String SET = "SET"; + + public static String toOceanBaseType(String type, Integer length, Integer scale) { + switch (type.toUpperCase()) { + case BIT: + return BIT; + case BOOLEAN: + case BOOL: + return BOOLEAN; + case TINYINT: + case TINYINT_UNSIGNED: + case TINYINT_UNSIGNED_ZEROFILL: + return TINYINT; + case SMALLINT: + case SMALLINT_UNSIGNED: + case SMALLINT_UNSIGNED_ZEROFILL: + return SMALLINT; + case MEDIUMINT: + case MEDIUMINT_UNSIGNED: + case MEDIUMINT_UNSIGNED_ZEROFILL: + return MEDIUMINT; + case INT: + case INTEGER: + case INT_UNSIGNED: + case INT_UNSIGNED_ZEROFILL: + case INTEGER_UNSIGNED: + case INTEGER_UNSIGNED_ZEROFILL: + return INT; + case BIGINT: + case BIGINT_UNSIGNED: + case BIGINT_UNSIGNED_ZEROFILL: + case SERIAL: + return BIGINT; + case REAL: + case REAL_UNSIGNED: + case REAL_UNSIGNED_ZEROFILL: + case FLOAT: + case FLOAT_UNSIGNED: + case FLOAT_UNSIGNED_ZEROFILL: + return FLOAT; + case DOUBLE: + case DOUBLE_UNSIGNED: + case DOUBLE_UNSIGNED_ZEROFILL: + case DOUBLE_PRECISION: + case DOUBLE_PRECISION_UNSIGNED: + case DOUBLE_PRECISION_UNSIGNED_ZEROFILL: + return DOUBLE; + case NUMERIC: + case NUMERIC_UNSIGNED: + case NUMERIC_UNSIGNED_ZEROFILL: + case FIXED: + case FIXED_UNSIGNED: + case FIXED_UNSIGNED_ZEROFILL: + case DECIMAL: + case DECIMAL_UNSIGNED: + case DECIMAL_UNSIGNED_ZEROFILL: + return DECIMAL + "(" + length + "," + scale + ")"; + case CHAR: + return CHAR + "(" + length + ")"; + case VARCHAR: + return VARCHAR + "(" + length + ")"; + case TINYTEXT: + case MEDIUMTEXT: + case TEXT: + case LONGTEXT: + return TEXT; + case DATE: + return DATE; + case TIME: + return TIME + "(" + scale + ")"; + case DATETIME: + return DATETIME + "(" + scale + ")"; + case TIMESTAMP: + return TIMESTAMP + "(" + scale + ")"; + case YEAR: + return YEAR; + case BINARY: + return BINARY + "(" + length + ")"; + case VARBINARY: + return VARBINARY + "(" + length + ")"; + case TINYBLOB: + case MEDIUMBLOB: + case BLOB: + case LONGBLOB: + return BLOB; + case JSON: + return JSON; + case ENUM: + case SET: + return type; // ENUM and SET types are kept as is + default: + throw new UnsupportedOperationException("Unsupported MySQL type: " + type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java new file mode 100644 index 0000000..fb050d6 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDatabaseSync.java @@ -0,0 +1,257 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.oracle.OracleSource; +import org.apache.flink.cdc.connectors.oracle.source.OracleSourceBuilder; +import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceOptions; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class OracleDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(OracleDatabaseSync.class); + + private static final String JDBC_URL = "jdbc:oracle:thin:@%s:%d:%s"; + private static final String PDB_KEY = "debezium.database.pdb.name"; + + public OracleDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("oracle.jdbc.driver.OracleDriver"); + } catch (ClassNotFoundException ex) { + LOG.warn( + "can not found class oracle.jdbc.driver.OracleDriver, use class oracle.jdbc.OracleDriver"); + try { + Class.forName("oracle.jdbc.OracleDriver"); + } catch (Exception e) { + throw new SQLException( + "No suitable driver found, can not found class oracle.jdbc.driver.OracleDriver and oracle.jdbc.OracleDriver"); + } + } + } + + @Override + public Connection getConnection() throws SQLException { + String jdbcUrl; + if (!StringUtils.isNullOrWhitespaceOnly(config.get(OracleSourceOptions.URL))) { + jdbcUrl = config.get(OracleSourceOptions.URL); + } else { + jdbcUrl = + String.format( + JDBC_URL, + config.get(OracleSourceOptions.HOSTNAME), + config.get(OracleSourceOptions.PORT), + config.get(OracleSourceOptions.DATABASE_NAME)); + } + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(OracleSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(OracleSourceOptions.PASSWORD)); + pro.put("remarksReporting", "true"); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(OracleSourceOptions.DATABASE_NAME); + String schemaName = config.get(OracleSourceOptions.SCHEMA_NAME); + + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + setSessionToPdb(conn); + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(databaseName, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new OracleSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + private void setSessionToPdb(Connection conn) throws SQLException { + String pdbName = null; + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + if (key.equals(PDB_KEY)) { + pdbName = entry.getValue(); + break; + } + } + if (!StringUtils.isNullOrWhitespaceOnly(pdbName)) { + LOG.info("Found pdb name in config, set session to pdb to {}", pdbName); + try (Statement statement = conn.createStatement()) { + statement.execute("alter session set container=" + pdbName); + } + } + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + Properties debeziumProperties = new Properties(); + String databaseName = config.get(OracleSourceOptions.DATABASE_NAME); + String schemaName = config.get(OracleSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in oracle is required"); + Preconditions.checkNotNull(schemaName, "schema-name in oracle is required"); + String tableName = config.get(OracleSourceOptions.TABLE_NAME); + // When debezium incrementally reads, it will be judged based on regexp_like. + // When the regular length exceeds 512, an error will be reported, + // like ORA-12733: regular expression too long + if (tableName.length() > 512) { + tableName = StringUtils.isNullOrWhitespaceOnly(includingTables) ? ".*" : tableName; + } + + String url = config.get(OracleSourceOptions.URL); + String hostname = config.get(OracleSourceOptions.HOSTNAME); + Integer port = config.get(OracleSourceOptions.PORT); + String username = config.get(OracleSourceOptions.USERNAME); + String password = config.get(OracleSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(OracleSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + // date to string + debeziumProperties.putAll(OracleDateConverter.DEFAULT_PROPS); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(OracleSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + OracleSourceBuilder.OracleIncrementalSource.builder() + .hostname(hostname) + .url(url) + .port(port) + .databaseList(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .includeSchemaChanges(true) + .startupOptions(startupOptions) + .deserializer(schema) + .debeziumProperties(debeziumProperties) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "Oracle IncrSource"); + } else { + DebeziumSourceFunction oracleSource = + OracleSource.builder() + .url(url) + .hostname(hostname) + .port(port) + .username(username) + .password(password) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(oracleSource, "Oracle Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(OracleSourceOptions.SCHEMA_NAME); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java new file mode 100644 index 0000000..c3776a2 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleDateConverter.java @@ -0,0 +1,152 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import oracle.sql.TIMESTAMP; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class OracleDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(OracleDateConverter.class); + private static final Pattern TO_DATE = + Pattern.compile("TO_DATE\\('(.*)',[ ]*'(.*)'\\)", Pattern.CASE_INSENSITIVE); + private static final Pattern TO_TIMESTAMP = + Pattern.compile("TO_TIMESTAMP\\('(.*)'\\)", Pattern.CASE_INSENSITIVE); + private static final Pattern TIMESTAMP_OR_DATE_REGEX = + Pattern.compile("^TIMESTAMP[(]\\d[)]$|^DATE$", Pattern.CASE_INSENSITIVE); + private ZoneId timestampZoneId = ZoneId.systemDefault(); + public static final Properties DEFAULT_PROPS = new Properties(); + private static final String DATETIME_PATTERN = DatabaseSyncConfig.DATE_TIME_FORMAT; + private static final String DATETIMEV2_PATTERN = DatabaseSyncConfig.DATETIME_MICRO_FORMAT; + private final DateTimeFormatter dateTimeV2Formatter = + DateTimeFormatter.ofPattern(DATETIMEV2_PATTERN); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, "oracleDate"); + DEFAULT_PROPS.setProperty( + "oracleDate.type", + "com.oceanbase.connector.flink.tools.cdc.oracle.OracleDateConverter"); + } + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern(DatabaseSyncConfig.DATE_TIME_FORMAT) + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .toFormatter(); + + @Override + public void configure(Properties props) {} + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String typeName = column.typeName(); + if (TIMESTAMP_OR_DATE_REGEX.matcher(typeName).matches()) { + registration.register( + SchemaBuilder.string().optional(), + value -> { + if (value == null) { + if (column.isOptional()) { + return null; + } else if (column.hasDefaultValue()) { + return column.defaultValue(); + } else { + return null; + } + } + + if (value instanceof String) { + return convertStringTimestamp((String) value); + } + if (value instanceof Timestamp) { + return dateTimeV2Formatter.format( + ((Timestamp) value).toLocalDateTime()); + } + + // oracle timestamp + try { + if (value instanceof TIMESTAMP) { + return dateTimeV2Formatter.format( + ((TIMESTAMP) value).timestampValue().toLocalDateTime()); + } + } catch (SQLException ex) { + log.error("convert timestamp failed, values is {}", value); + } + + return null; + }); + } + } + + private String convertStringTimestamp(String data) { + LocalDateTime dateTime; + + final Matcher toTimestampMatcher = TO_TIMESTAMP.matcher(data); + if (toTimestampMatcher.matches()) { + String dateText = toTimestampMatcher.group(1); + dateTime = + LocalDateTime.from( + TIMESTAMP_FORMATTER.parse(completeMilliseconds(dateText.trim()))); + return dateTimeV2Formatter.format(dateTime.atZone(timestampZoneId)); + } + + final Matcher toDateMatcher = TO_DATE.matcher(data); + if (toDateMatcher.matches()) { + String date = toDateMatcher.group(1); + dateTime = + LocalDateTime.from( + TIMESTAMP_FORMATTER.parse(completeMilliseconds(date.trim()))); + return dateTimeV2Formatter.format(dateTime.atZone(timestampZoneId)); + } + return null; + } + + private String completeMilliseconds(String stringValue) { + if (stringValue.length() == DATETIMEV2_PATTERN.length()) { + return stringValue; + } + StringBuilder sb = new StringBuilder(stringValue); + if (stringValue.length() == DATETIME_PATTERN.length()) { + sb.append("."); + } + while (sb.toString().length() < DATETIMEV2_PATTERN.length()) { + sb.append(0); + } + return sb.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java new file mode 100644 index 0000000..d82ffbf --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleSchema.java @@ -0,0 +1,54 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.catalog.FieldSchema; +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; +import java.sql.SQLException; +import java.util.LinkedHashMap; + +public class OracleSchema extends JdbcSourceSchema { + + public OracleSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return OracleType.toOceanBaseType(fieldType, precision, scale); + } + + @Override + public LinkedHashMap getColumnInfo( + DatabaseMetaData metaData, String databaseName, String schemaName, String tableName) + throws SQLException { + // Oracle permits table names to include special characters such as /, + // etc., as in 'A/B'. + // When attempting to fetch column information for `A/B` via JDBC, + // it may throw an ORA-01424 error. + // Hence, we substitute `/` with '_' to address the issue. + return super.getColumnInfo(metaData, databaseName, schemaName, tableName.replace("/", "_")); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java new file mode 100644 index 0000000..dd75bf7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/oracle/OracleType.java @@ -0,0 +1,106 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.tools.cdc.oracle; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class OracleType { + private static final String VARCHAR2 = "VARCHAR2"; + private static final String NVARCHAR2 = "NVARCHAR2"; + private static final String NUMBER = "NUMBER"; + private static final String FLOAT = "FLOAT"; + private static final String LONG = "LONG"; + private static final String DATE = "DATE"; + private static final String BINARY_FLOAT = "BINARY_FLOAT"; + private static final String BINARY_DOUBLE = "BINARY_DOUBLE"; + private static final String TIMESTAMP = "TIMESTAMP"; + private static final String INTERVAL = "INTERVAL"; + private static final String RAW = "RAW"; + private static final String LONG_RAW = "LONG RAW"; + private static final String CHAR = "CHAR"; + private static final String NCHAR = "NCHAR"; + private static final String CLOB = "CLOB"; + private static final String NCLOB = "NCLOB"; + private static final String BLOB = "BLOB"; + private static final String BFILE = "BFILE"; + private static final String XMLTYPE = "XMLTYPE"; + + public static String toOceanBaseType(String oracleType, Integer precision, Integer scale) { + oracleType = oracleType.toUpperCase(); + if (oracleType.startsWith(INTERVAL)) { + oracleType = oracleType.substring(0, 8); + } else if (oracleType.startsWith(TIMESTAMP)) { + return String.format("%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale, 6)); + } + switch (oracleType) { + case NUMBER: + if (scale <= 0) { + precision -= scale; + if (precision < 3) { + return OceanBaseType.TINYINT; + } else if (precision < 5) { + return OceanBaseType.SMALLINT; + } else if (precision < 10) { + return OceanBaseType.INT; + } else if (precision < 19) { + return OceanBaseType.BIGINT; + } else { + return OceanBaseType.LARGEINT; + } + } + // scale > 0 + if (precision < scale) { + precision = scale; + } + return precision != null && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.TIMESTAMP, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case FLOAT: + return OceanBaseType.DOUBLE; + case DATE: + // can save date and time with second precision + return OceanBaseType.DATE; + case CHAR: + case VARCHAR2: + case NCHAR: + case NVARCHAR2: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case LONG: + case RAW: + case LONG_RAW: + case INTERVAL: + case BLOB: + case CLOB: + case NCLOB: + case XMLTYPE: + return OceanBaseType.TEXT; + case BFILE: + case BINARY_FLOAT: + case BINARY_DOUBLE: + default: + throw new UnsupportedOperationException("Unsupported Oracle Type: " + oracleType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java new file mode 100644 index 0000000..dc95e39 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDatabaseSync.java @@ -0,0 +1,247 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.postgres.PostgreSQLSource; +import org.apache.flink.cdc.connectors.postgres.source.PostgresSourceBuilder; +import org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.DECODING_PLUGIN_NAME; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.SLOT_NAME; + +public class PostgresDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(PostgresDatabaseSync.class); + + private static final String JDBC_URL = "jdbc:postgresql://%s:%d/%s?"; + + public PostgresDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("org.postgresql.Driver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class org.postgresql.Driver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(PostgresSourceOptions.HOSTNAME), + config.get(PostgresSourceOptions.PG_PORT), + config.get(PostgresSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(PostgresSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(PostgresSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(PostgresSourceOptions.DATABASE_NAME); + String schemaName = config.get(PostgresSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables( + databaseName, + schemaName, + "%", + new String[] {"TABLE", "PARTITIONED TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new PostgresSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(PostgresSourceOptions.DATABASE_NAME); + String schemaName = config.get(PostgresSourceOptions.SCHEMA_NAME); + String slotName = config.get(SLOT_NAME); + Preconditions.checkNotNull(databaseName, "database-name in postgres is required"); + Preconditions.checkNotNull(schemaName, "schema-name in postgres is required"); + Preconditions.checkNotNull(slotName, "slot.name in postgres is required"); + + String tableName = config.get(PostgresSourceOptions.TABLE_NAME); + String hostname = config.get(PostgresSourceOptions.HOSTNAME); + Integer port = config.get(PostgresSourceOptions.PG_PORT); + String username = config.get(PostgresSourceOptions.USERNAME); + String password = config.get(PostgresSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(PostgresSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(PostgresDateConverter.DEFAULT_PROPS); + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + PostgresSourceBuilder.PostgresIncrementalSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .deserializer(schema) + .slotName(slotName) + .decodingPluginName(config.get(DECODING_PLUGIN_NAME)) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .heartbeatInterval(config.get(HEARTBEAT_INTERVAL)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "Postgres IncrSource"); + } else { + DebeziumSourceFunction postgresSource = + PostgreSQLSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .schemaList(schemaName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .deserializer(schema) + .slotName(slotName) + .decodingPluginName(config.get(DECODING_PLUGIN_NAME)) + .build(); + return env.addSource(postgresSource, "Postgres Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(PostgresSourceOptions.SCHEMA_NAME); + } + + @Override + protected String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + + if (!initialJdbcUrl.startsWith("?")) { + return super.getJdbcUrlTemplate(initialJdbcUrl, jdbcProperties); + } + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + int recordIndex = 0; + for (Map.Entry entry : jdbcProperties.entrySet()) { + jdbcUrlBuilder.append(entry.getKey()).append("=").append(entry.getValue()); + if (recordIndex < jdbcProperties.size() - 1) { + jdbcUrlBuilder.append("&"); + recordIndex++; + } + } + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java new file mode 100644 index 0000000..fe0152c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresDateConverter.java @@ -0,0 +1,133 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class PostgresDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(PostgresDateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.postgres.PostgresDateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.TIME.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTime; + } + if (DatabaseSyncConfig.TIMESTAMP.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertTimestamp; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private String convertDate(Object input) { + if (input instanceof LocalDate) { + return dateFormatter.format((LocalDate) input); + } else if (input instanceof Integer) { + LocalDate date = LocalDate.ofEpochDay((Integer) input); + return dateFormatter.format(date); + } else if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } + + private String convertTime(Object input) { + if (input instanceof String) { + return input.toString(); + } + return null; + } + + private String convertTimestamp(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } else if (input instanceof Instant) { + LocalDateTime ldt = LocalDateTime.ofInstant(((Instant) input), ZoneOffset.UTC); + return timestampFormatter.format(ldt); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java new file mode 100644 index 0000000..d637555 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresSchema.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class PostgresSchema extends JdbcSourceSchema { + + public PostgresSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return PostgresType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java new file mode 100644 index 0000000..1cd42aa --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/postgres/PostgresType.java @@ -0,0 +1,159 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.postgres; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +import org.apache.flink.util.Preconditions; + +public class PostgresType { + private static final String INT2 = "int2"; + private static final String SMALLSERIAL = "smallserial"; + private static final String INT4 = "int4"; + private static final String SERIAL = "serial"; + private static final String INT8 = "int8"; + private static final String BIGSERIAL = "bigserial"; + private static final String NUMERIC = "numeric"; + private static final String FLOAT4 = "float4"; + private static final String FLOAT8 = "float8"; + private static final String BPCHAR = "bpchar"; + private static final String TIMESTAMP = "timestamp"; + private static final String TIMESTAMPTZ = "timestamptz"; + private static final String DATE = "date"; + private static final String BOOL = "bool"; + private static final String BIT = "bit"; + private static final String POINT = "point"; + private static final String LINE = "line"; + private static final String LSEG = "lseg"; + private static final String BOX = "box"; + private static final String PATH = "path"; + private static final String POLYGON = "polygon"; + private static final String CIRCLE = "circle"; + private static final String VARCHAR = "varchar"; + private static final String TEXT = "text"; + private static final String TIME = "time"; + private static final String TIMETZ = "timetz"; + private static final String INTERVAL = "interval"; + private static final String CIDR = "cidr"; + private static final String INET = "inet"; + private static final String MACADDR = "macaddr"; + private static final String VARBIT = "varbit"; + private static final String UUID = "uuid"; + private static final String BYTEA = "bytea"; + private static final String JSON = "json"; + private static final String JSONB = "jsonb"; + private static final String _INT2 = "_int2"; + private static final String _INT4 = "_int4"; + private static final String _INT8 = "_int8"; + private static final String _FLOAT4 = "_float4"; + private static final String _FLOAT8 = "_float8"; + private static final String _DATE = "_date"; + private static final String _TIMESTAMP = "_timestamp"; + private static final String _BOOL = "_bool"; + private static final String _TEXT = "_text"; + + public static String toOceanBaseType(String postgresType, Integer precision, Integer scale) { + postgresType = postgresType.toLowerCase(); + if (postgresType.startsWith("_")) { + return OceanBaseType.VARCHAR; + } + switch (postgresType) { + case INT2: + case SMALLSERIAL: + return OceanBaseType.TINYINT; + case INT4: + case SERIAL: + return OceanBaseType.INT; + case INT8: + case BIGSERIAL: + return OceanBaseType.BIGINT; + case NUMERIC: + return precision != null && precision > 0 && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.DECIMAL, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case FLOAT4: + return OceanBaseType.FLOAT; + case FLOAT8: + return OceanBaseType.DOUBLE; + case TIMESTAMP: + case TIMESTAMPTZ: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case DATE: + return OceanBaseType.DATE; + case BOOL: + return OceanBaseType.BOOLEAN; + case BIT: + return precision == 1 ? OceanBaseType.BOOLEAN : OceanBaseType.VARCHAR; + case BPCHAR: + case VARCHAR: + Preconditions.checkNotNull(precision); + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case POINT: + case LINE: + case LSEG: + case BOX: + case PATH: + case POLYGON: + case CIRCLE: + case TEXT: + case TIME: + case TIMETZ: + case INTERVAL: + case CIDR: + case INET: + case MACADDR: + case VARBIT: + case UUID: + case BYTEA: + return OceanBaseType.VARCHAR; + case JSON: + case JSONB: + return OceanBaseType.JSONB; + /* Compatible with oceanbase1.2 array type can only be used in dup table, + and then converted to array in the next version + case _BOOL: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.BOOLEAN); + case _INT2: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.TINYINT); + case _INT4: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.INT); + case _INT8: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.BIGINT); + case _FLOAT4: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.FLOAT); + case _FLOAT8: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DOUBLE); + case _TEXT: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.STRING); + case _DATE: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DATE_V2); + case _TIMESTAMP: + return String.format("%s<%s>", OceanBaseType.ARRAY, OceanBaseType.DATETIME_V2); + **/ + default: + throw new UnsupportedOperationException( + "Unsupported Postgres Type: " + postgresType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java new file mode 100644 index 0000000..d44dcb7 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java @@ -0,0 +1,224 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; +import com.oceanbase.connector.flink.tools.cdc.SourceSchema; +import com.oceanbase.connector.flink.tools.cdc.deserialize.OceanBaseJsonDebeziumDeserializationSchema; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; +import org.apache.flink.cdc.connectors.base.options.SourceOptions; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.flink.cdc.connectors.sqlserver.SqlServerSource; +import org.apache.flink.cdc.connectors.sqlserver.source.SqlServerSourceBuilder; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.DebeziumSourceFunction; +import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; + +public class SqlServerDatabaseSync extends DatabaseSync { + private static final Logger LOG = LoggerFactory.getLogger(SqlServerDatabaseSync.class); + private static final String JDBC_URL = "jdbc:sqlserver://%s:%d;database=%s;"; + private static final String PORT = "port"; + + public SqlServerDatabaseSync() throws SQLException { + super(); + } + + @Override + public void registerDriver() throws SQLException { + try { + Class.forName("com.microsoft.sqlserver.jdbc.SQLServerDriver"); + } catch (ClassNotFoundException ex) { + throw new SQLException( + "No suitable driver found, can not found class com.microsoft.sqlserver.jdbc.SQLServerDriver"); + } + } + + @Override + public Connection getConnection() throws SQLException { + Properties jdbcProperties = getJdbcProperties(); + String jdbcUrlTemplate = getJdbcUrlTemplate(JDBC_URL, jdbcProperties); + String jdbcUrl = + String.format( + jdbcUrlTemplate, + config.get(JdbcSourceOptions.HOSTNAME), + config.getInteger(PORT, 1433), + config.get(JdbcSourceOptions.DATABASE_NAME)); + Properties pro = new Properties(); + pro.setProperty(DatabaseSyncConfig.USER, config.get(JdbcSourceOptions.USERNAME)); + pro.setProperty(DatabaseSyncConfig.PASSWORD, config.get(JdbcSourceOptions.PASSWORD)); + return DriverManager.getConnection(jdbcUrl, pro); + } + + @Override + public List getSchemaList() throws Exception { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + List schemaList = new ArrayList<>(); + LOG.info("database-name {}, schema-name {}", databaseName, schemaName); + try (Connection conn = getConnection()) { + DatabaseMetaData metaData = conn.getMetaData(); + try (ResultSet tables = + metaData.getTables(databaseName, schemaName, "%", new String[] {"TABLE"})) { + while (tables.next()) { + String tableName = tables.getString(DatabaseSyncConfig.TABLE_NAME); + String tableComment = tables.getString(DatabaseSyncConfig.REMARKS); + if (!isSyncNeeded(tableName)) { + continue; + } + SourceSchema sourceSchema = + new SqlServerSchema( + metaData, databaseName, schemaName, tableName, tableComment); + schemaList.add(sourceSchema); + } + } + } + return schemaList; + } + + @Override + public DataStreamSource buildCdcSource(StreamExecutionEnvironment env) { + String databaseName = config.get(JdbcSourceOptions.DATABASE_NAME); + String schemaName = config.get(JdbcSourceOptions.SCHEMA_NAME); + Preconditions.checkNotNull(databaseName, "database-name in sqlserver is required"); + Preconditions.checkNotNull(schemaName, "schema-name in sqlserver is required"); + + String tableName = config.get(JdbcSourceOptions.TABLE_NAME); + String hostname = config.get(JdbcSourceOptions.HOSTNAME); + int port = config.getInteger(PORT, 1433); + String username = config.get(JdbcSourceOptions.USERNAME); + String password = config.get(JdbcSourceOptions.PASSWORD); + + StartupOptions startupOptions = StartupOptions.initial(); + String startupMode = config.get(JdbcSourceOptions.SCAN_STARTUP_MODE); + if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_INITIAL.equalsIgnoreCase(startupMode)) { + startupOptions = StartupOptions.initial(); + } else if (DatabaseSyncConfig.SCAN_STARTUP_MODE_VALUE_LATEST_OFFSET.equalsIgnoreCase( + startupMode)) { + startupOptions = StartupOptions.latest(); + } + + // debezium properties set + Properties debeziumProperties = new Properties(); + debeziumProperties.putAll(SqlServerDateConverter.DEFAULT_PROPS); + debeziumProperties.put(DatabaseSyncConfig.DECIMAL_HANDLING_MODE, "string"); + + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX)) { + debeziumProperties.put( + key.substring(DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX.length()), value); + } + } + + DebeziumDeserializationSchema schema; + if (ignoreDefaultValue) { + schema = new OceanBaseJsonDebeziumDeserializationSchema(); + } else { + Map customConverterConfigs = new HashMap<>(); + customConverterConfigs.put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric"); + schema = new JsonDebeziumDeserializationSchema(false, customConverterConfigs); + } + + if (config.getBoolean(SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED, false)) { + JdbcIncrementalSource incrSource = + SqlServerSourceBuilder.SqlServerIncrementalSource.builder() + .hostname(hostname) + .port(port) + .databaseList(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .startupOptions(startupOptions) + .deserializer(schema) + .includeSchemaChanges(true) + .debeziumProperties(debeziumProperties) + .splitSize(config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE)) + .splitMetaGroupSize(config.get(CHUNK_META_GROUP_SIZE)) + .fetchSize(config.get(SCAN_SNAPSHOT_FETCH_SIZE)) + .connectTimeout(config.get(CONNECT_TIMEOUT)) + .connectionPoolSize(config.get(CONNECTION_POOL_SIZE)) + .connectMaxRetries(config.get(CONNECT_MAX_RETRIES)) + .distributionFactorUpper( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND)) + .distributionFactorLower( + config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND)) + .build(); + return env.fromSource( + incrSource, WatermarkStrategy.noWatermarks(), "SqlServer IncrSource"); + } else { + DebeziumSourceFunction sqlServerSource = + SqlServerSource.builder() + .hostname(hostname) + .port(port) + .database(databaseName) + .tableList(tableName) + .username(username) + .password(password) + .debeziumProperties(debeziumProperties) + .startupOptions(startupOptions) + .deserializer(schema) + .build(); + return env.addSource(sqlServerSource, "SqlServer Source"); + } + } + + @Override + public String getTableListPrefix() { + return config.get(JdbcSourceOptions.SCHEMA_NAME); + } + + @Override + public String getJdbcUrlTemplate(String initialJdbcUrl, Properties jdbcProperties) { + StringBuilder jdbcUrlBuilder = new StringBuilder(initialJdbcUrl); + jdbcProperties.forEach( + (key, value) -> jdbcUrlBuilder.append(key).append("=").append(value).append(";")); + return jdbcUrlBuilder.toString(); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java new file mode 100644 index 0000000..b29034c --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerDateConverter.java @@ -0,0 +1,112 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.cdc.connectors.shaded.org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.spi.converter.CustomConverter; +import io.debezium.spi.converter.RelationalColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.DateTimeException; +import java.time.format.DateTimeFormatter; +import java.util.Properties; +import java.util.function.Consumer; + +public class SqlServerDateConverter implements CustomConverter { + private static final Logger log = LoggerFactory.getLogger(SqlServerDateConverter.class); + private DateTimeFormatter dateFormatter = DateTimeFormatter.ISO_DATE; + private DateTimeFormatter timestampFormatter = DateTimeFormatter.ISO_DATE_TIME; + + public static final Properties DEFAULT_PROPS = new Properties(); + + static { + DEFAULT_PROPS.setProperty(DatabaseSyncConfig.CONVERTERS, DatabaseSyncConfig.DATE); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_TYPE, + "com.oceanbase.connector.flink.tools.cdc.sqlserver.SqlServerDateConverter"); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_DATE, DatabaseSyncConfig.YEAR_MONTH_DAY_FORMAT); + DEFAULT_PROPS.setProperty( + DatabaseSyncConfig.DATE_FORMAT_TIMESTAMP, DatabaseSyncConfig.DATETIME_MICRO_FORMAT); + } + + @Override + public void configure(Properties props) { + readProps( + props, + DatabaseSyncConfig.FORMAT_DATE, + p -> dateFormatter = DateTimeFormatter.ofPattern(p)); + readProps( + props, + DatabaseSyncConfig.FORMAT_TIMESTAMP, + p -> timestampFormatter = DateTimeFormatter.ofPattern(p)); + } + + private void readProps(Properties properties, String settingKey, Consumer callback) { + String settingValue = (String) properties.get(settingKey); + if (settingValue == null || settingValue.length() == 0) { + return; + } + try { + callback.accept(settingValue.trim()); + } catch (IllegalArgumentException | DateTimeException e) { + log.error("setting {} is illegal:{}", settingKey, settingValue); + throw e; + } + } + + @Override + public void converterFor( + RelationalColumn column, ConverterRegistration registration) { + String sqlType = column.typeName().toUpperCase(); + SchemaBuilder schemaBuilder = null; + Converter converter = null; + if (DatabaseSyncConfig.UPPERCASE_DATE.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDate; + } + if (DatabaseSyncConfig.SMALLDATETIME.equals(sqlType) + || DatabaseSyncConfig.DATETIME.equals(sqlType) + || DatabaseSyncConfig.DATETIME2.equals(sqlType)) { + schemaBuilder = SchemaBuilder.string().optional(); + converter = this::convertDateTime; + } + if (schemaBuilder != null) { + registration.register(schemaBuilder, converter); + } + } + + private Object convertDateTime(Object input) { + if (input instanceof Timestamp) { + return timestampFormatter.format(((Timestamp) input).toLocalDateTime()); + } + return null; + } + + private String convertDate(Object input) { + if (input instanceof Date) { + return dateFormatter.format(((Date) input).toLocalDate()); + } + return null; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java new file mode 100644 index 0000000..9706819 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerSchema.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.cdc.JdbcSourceSchema; + +import java.sql.DatabaseMetaData; + +public class SqlServerSchema extends JdbcSourceSchema { + + public SqlServerSchema( + DatabaseMetaData metaData, + String databaseName, + String schemaName, + String tableName, + String tableComment) + throws Exception { + super(metaData, databaseName, schemaName, tableName, tableComment); + } + + @Override + public String convertToOceanBaseType(String fieldType, Integer precision, Integer scale) { + return SqlServerType.toOceanBaseType(fieldType, precision, scale); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java new file mode 100644 index 0000000..3e1b044 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/tools/cdc/sqlserver/SqlServerType.java @@ -0,0 +1,115 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.connector.flink.tools.cdc.sqlserver; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseType; + +public class SqlServerType { + private static final String BIT = "bit"; + private static final String TINYINT = "tinyint"; + private static final String SMALLINT = "smallint"; + private static final String INT = "int"; + private static final String BIGINT = "bigint"; + private static final String REAL = "real"; + private static final String FLOAT = "float"; + private static final String MONEY = "money"; + private static final String SMALLMONEY = "smallmoney"; + private static final String DECIMAL = "decimal"; + private static final String NUMERIC = "numeric"; + private static final String DATE = "date"; + private static final String DATETIME = "datetime"; + private static final String DATETIME2 = "datetime2"; + private static final String SMALLDATETIME = "smalldatetime"; + private static final String CHAR = "char"; + private static final String VARCHAR = "varchar"; + private static final String NCHAR = "nchar"; + private static final String NVARCHAR = "nvarchar"; + private static final String TEXT = "text"; + private static final String NTEXT = "ntext"; + private static final String XML = "xml"; + private static final String UNIQUEIDENTIFIER = "uniqueidentifier"; + private static final String TIME = "time"; + private static final String TIMESTAMP = "timestamp"; + private static final String DATETIMEOFFSET = "datetimeoffset"; + private static final String IMAGE = "image"; + private static final String BINARY = "binary"; + private static final String VARBINARY = "varbinary"; + + public static String toOceanBaseType( + String originSqlServerType, Integer precision, Integer scale) { + originSqlServerType = originSqlServerType.toLowerCase(); + // For sqlserver IDENTITY type, such as 'INT IDENTITY' + // originSqlServerType is "int identity", so we only get "int". + String sqlServerType = originSqlServerType.split(" ")[0]; + switch (sqlServerType) { + case BIT: + return OceanBaseType.BOOLEAN; + case TINYINT: + return OceanBaseType.TINYINT; + case SMALLINT: + return OceanBaseType.SMALLINT; + case INT: + return OceanBaseType.INT; + case BIGINT: + return OceanBaseType.BIGINT; + case REAL: + return OceanBaseType.FLOAT; + case FLOAT: + return OceanBaseType.DOUBLE; + case MONEY: + return String.format("%s(%s,%s)", OceanBaseType.DATETIME, 19, 4); + case SMALLMONEY: + return String.format("%s(%s,%s)", OceanBaseType.DATETIME, 10, 4); + case DECIMAL: + case NUMERIC: + return precision != null && precision > 0 && precision <= 38 + ? String.format( + "%s(%s,%s)", + OceanBaseType.DATETIME, + precision, + scale != null && scale >= 0 ? scale : 0) + : OceanBaseType.VARCHAR; + case DATE: + return OceanBaseType.DATE; + case DATETIME: + case DATETIME2: + case SMALLDATETIME: + return String.format( + "%s(%s)", OceanBaseType.TIMESTAMP, Math.min(scale == null ? 0 : scale, 6)); + case CHAR: + case VARCHAR: + case NCHAR: + case NVARCHAR: + return precision * 3 > 65533 + ? OceanBaseType.VARCHAR + : String.format("%s(%s)", OceanBaseType.VARCHAR, precision * 3); + case TEXT: + case NTEXT: + case TIME: + case DATETIMEOFFSET: + case TIMESTAMP: + case UNIQUEIDENTIFIER: + case BINARY: + case VARBINARY: + case XML: + return OceanBaseType.TEXT; + default: + throw new UnsupportedOperationException( + "Unsupported SqlServer Type: " + sqlServerType); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java new file mode 100644 index 0000000..e196ca9 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/main/java/com/oceanbase/connector/flink/utils/OceanBaseToolsJdbcUtils.java @@ -0,0 +1,133 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.connector.flink.utils; + +import com.oceanbase.connector.flink.tools.catalog.OceanBaseSchemaFactory; +import com.oceanbase.connector.flink.tools.catalog.TableSchema; + +import org.apache.flink.util.function.SupplierWithException; + +import org.apache.commons.compress.utils.Lists; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Predicate; + +import static org.apache.flink.util.Preconditions.checkArgument; + +public class OceanBaseToolsJdbcUtils extends OceanBaseJdbcUtils { + private static final List builtinDatabases = + Collections.singletonList("information_schema"); + + public static List listDatabases( + SupplierWithException connectionSupplier) { + return extractColumnValuesBySQL( + connectionSupplier, + "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA`;", + 1, + dbName -> !builtinDatabases.contains(dbName)); + } + + public static boolean databaseExists( + String database, SupplierWithException connectionSupplier) { + checkArgument(!org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly(database)); + return listDatabases(connectionSupplier).contains(database); + } + + public static void createDatabase( + String database, SupplierWithException connectionSupplier) { + execute(connectionSupplier, String.format("CREATE DATABASE IF NOT EXISTS %s", database)); + } + + public static boolean tableExists( + String database, + String table, + SupplierWithException connectionSupplier) { + return databaseExists(database, connectionSupplier) + && listTables(database, connectionSupplier).contains(table); + } + + public static List listTables( + String databaseName, + SupplierWithException connectionSupplier) { + if (!databaseExists(databaseName, connectionSupplier)) { + throw new RuntimeException("database" + databaseName + " is not exists"); + } + return extractColumnValuesBySQL( + connectionSupplier, + "SELECT TABLE_NAME FROM information_schema.`TABLES` WHERE TABLE_SCHEMA = ?", + 1, + null, + databaseName); + } + + public static void createTable( + TableSchema schema, + SupplierWithException connectionSupplier) { + String ddl = buildCreateTableDDL(schema); + execute(connectionSupplier, ddl); + } + + public static void execute( + SupplierWithException connectionSupplier, String sql) { + try (Connection connection = connectionSupplier.get(); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } catch (Exception e) { + throw new RuntimeException( + String.format("SQL query could not be executed: %s", sql), e); + } + } + + public static List extractColumnValuesBySQL( + SupplierWithException connectionSupplier, + String sql, + int columnIndex, + Predicate filterFunc, + Object... params) { + List columnValues = Lists.newArrayList(); + try (Connection connection = connectionSupplier.get(); + PreparedStatement ps = connection.prepareStatement(sql)) { + if (Objects.nonNull(params) && params.length > 0) { + for (int i = 0; i < params.length; i++) { + ps.setObject(i + 1, params[i]); + } + } + try (ResultSet rs = ps.executeQuery()) { + while (rs.next()) { + String columnValue = rs.getString(columnIndex); + if (filterFunc == null || filterFunc.test(columnValue)) { + columnValues.add(columnValue); + } + } + } + return columnValues; + } catch (Exception e) { + throw new RuntimeException( + String.format("The following SQL query could not be executed: %s", sql), e); + } + } + + public static String buildCreateTableDDL(TableSchema schema) { + return OceanBaseSchemaFactory.generateCreateTableDDL(schema); + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java new file mode 100644 index 0000000..fa0d2eb --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcMysqlSyncDatabaseCase.java @@ -0,0 +1,222 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.OceanBaseMySQLTestBase; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSync; +import com.oceanbase.connector.flink.tools.cdc.mysql.MysqlDatabaseSync; + +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.shaded.org.apache.commons.io.IOUtils; +import org.testcontainers.utility.DockerLoggerFactory; + +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +public class CdcMysqlSyncDatabaseCase extends OceanBaseMySQLTestBase { + private static final Logger LOG = LoggerFactory.getLogger(CdcMysqlSyncDatabaseCase.class); + + private static final String MYSQL_HOST = "localhost"; + private static final Integer MYSQL_PORT = 3306; + private static final String MYSQL_USER_NAME = "root"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; + private static final String MYSQL_DATABASE = "mysql_cdc"; + private static final String MYSQL_TABLE_NAME = "test_history_text"; + static StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + @BeforeClass + public static void setup() { + CONFIG_SERVER.withLogConsumer(new Slf4jLogConsumer(LOG)).start(); + CONTAINER + .withEnv("OB_CONFIGSERVER_ADDRESS", getConfigServerAddress()) + .withLogConsumer(new Slf4jLogConsumer(LOG)) + .start(); + MYSQL_CONTAINER.start(); + } + + @AfterClass + public static void tearDown() { + Stream.of(MYSQL_CONTAINER).forEach(GenericContainer::stop); + } + + private static final MySqlContainer MYSQL_CONTAINER = + new MySqlContainer() + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("sql/cdc.sql") + // .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withExposedPorts(MYSQL_PORT) + .withDatabaseName(MYSQL_DATABASE) + .withPassword(MYSQL_USER_PASSWORD) + .withEnv("TZ", "Asia/Shanghai") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("mysql-docker-image"))); + + @Test + public void testCdcMysqlSyncOceanBase() throws Exception { + extractedCdcSync(); + checkResult(); + } + + private static void extractedCdcSync() throws Exception { + // env.setParallelism(1); + Map flinkMap = new HashMap<>(); + flinkMap.put("execution.checkpointing.interval", "10s"); + flinkMap.put("pipeline.operator-chaining", "false"); + flinkMap.put("parallelism.default", "1"); + + Configuration configuration = Configuration.fromMap(flinkMap); + env.configure(configuration); + + String tablePrefix = ""; + String tableSuffix = ""; + Map mysqlConfig = new HashMap<>(); + mysqlConfig.put(MySqlSourceOptions.DATABASE_NAME.key(), MYSQL_DATABASE); + mysqlConfig.put(MySqlSourceOptions.HOSTNAME.key(), MYSQL_HOST); + mysqlConfig.put( + MySqlSourceOptions.PORT.key(), + String.valueOf(MYSQL_CONTAINER.getMappedPort(MYSQL_PORT))); + mysqlConfig.put(MySqlSourceOptions.USERNAME.key(), MYSQL_USER_NAME); + mysqlConfig.put(MySqlSourceOptions.PASSWORD.key(), MYSQL_USER_PASSWORD); + // add jdbc properties for MySQL + mysqlConfig.put("jdbc.properties.use_ssl", "false"); + Configuration config = Configuration.fromMap(mysqlConfig); + + Map sinkConfig = new HashMap<>(); + sinkConfig.put(OceanBaseConnectorOptions.USERNAME.key(), CONTAINER.getUsername()); + sinkConfig.put(OceanBaseConnectorOptions.PASSWORD.key(), CONTAINER.getPassword()); + sinkConfig.put(OceanBaseConnectorOptions.URL.key(), CONTAINER.getJdbcUrl()); + sinkConfig.put("sink.enable-delete", "false"); + Configuration sinkConf = Configuration.fromMap(sinkConfig); + + String includingTables = "test.*"; + String excludingTables = ""; + boolean ignoreDefaultValue = false; + boolean ignoreIncompatible = false; + DatabaseSync databaseSync = new MysqlDatabaseSync(); + databaseSync + .setEnv(env) + .setDatabase(MYSQL_DATABASE) + .setConfig(config) + .setTablePrefix(tablePrefix) + .setTableSuffix(tableSuffix) + .setIncludingTables(includingTables) + .setExcludingTables(excludingTables) + .setIgnoreDefaultValue(ignoreDefaultValue) + .setSinkConfig(sinkConf) + .setCreateTableOnly(false) + .create(); + databaseSync.build(); + env.executeAsync(String.format("MySQL-Doris Database Sync: %s", MYSQL_DATABASE)); + checkResult(); + env.close(); + } + + static void checkResult() { + String sourceSql = String.format("select * from %s order by 1", MYSQL_TABLE_NAME); + String sinkSql = String.format("select * from %s order by 1", MYSQL_TABLE_NAME); + try (Statement sourceStatement = + getConnection( + getJdbcUrl( + MYSQL_HOST, + MYSQL_CONTAINER.getMappedPort(MYSQL_PORT), + MYSQL_DATABASE), + MYSQL_USER_NAME, + MYSQL_USER_PASSWORD) + .createStatement( + ResultSet.TYPE_SCROLL_INSENSITIVE, + ResultSet.CONCUR_READ_ONLY); + Statement sinkStatement = + getConnection( + CONTAINER.getJdbcUrl(), + CONTAINER.getUsername(), + CONTAINER.getPassword()) + .createStatement( + ResultSet.TYPE_SCROLL_INSENSITIVE, + ResultSet.CONCUR_READ_ONLY); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql)) { + Assertions.assertEquals( + sourceResultSet.getMetaData().getColumnCount(), + sinkResultSet.getMetaData().getColumnCount()); + while (sourceResultSet.next()) { + if (sinkResultSet.next()) { + for (String column : getFieldNames()) { + Object source = sourceResultSet.getObject(column); + Object sink = sinkResultSet.getObject(column); + if (!Objects.deepEquals(source, sink)) { + InputStream sourceAsciiStream = sourceResultSet.getBinaryStream(column); + InputStream sinkAsciiStream = sinkResultSet.getBinaryStream(column); + String sourceValue = + IOUtils.toString(sourceAsciiStream, StandardCharsets.UTF_8); + String sinkValue = + IOUtils.toString(sinkAsciiStream, StandardCharsets.UTF_8); + Assertions.assertEquals(sourceValue, sinkValue); + } + } + } + } + sourceResultSet.last(); + sinkResultSet.last(); + } catch (Exception e) { + throw new RuntimeException("Compare result error", e); + } + } + + static String[] getFieldNames() { + return new String[] { + "itemid", "clock", "value", "ns", + }; + } + + public static Connection getConnection(String jdbcUrl, String userName, String password) + throws SQLException { + return DriverManager.getConnection(jdbcUrl, userName, password); + } + + public static String getJdbcUrl(String host, Integer port, String schema) { + return "jdbc:mysql://" + + host + + ":" + + port + + "/" + + schema + + "?useUnicode=true&characterEncoding=UTF-8&useSSL=false"; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java new file mode 100644 index 0000000..6c7dce0 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/CdcToolsTest.java @@ -0,0 +1,94 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.tools.cdc.CdcTools; +import com.oceanbase.connector.flink.tools.cdc.DatabaseSyncConfig; + +import org.apache.flink.api.java.utils.MultipleParameterTool; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +public class CdcToolsTest { + + @Test + public void getConfigMapTest() { + MultipleParameterTool params = + MultipleParameterTool.fromArgs( + new String[] { + "--sink-conf", + "password=password", + "--sink-conf", + "username=username", + "--sink-conf", + "url=jdbc:mysql://127.0.0.1:2881 " + }); + Map sinkConf = CdcTools.getConfigMap(params, DatabaseSyncConfig.SINK_CONF); + + Map excepted = new HashMap<>(); + excepted.put("password", "password"); + excepted.put("username", "username"); + excepted.put("url", "jdbc:mysql://127.0.0.1:2881"); + Assert.assertEquals(sinkConf, excepted); + + Map mysqlConf = + CdcTools.getConfigMap(params, DatabaseSyncConfig.MYSQL_CONF); + Assert.assertNull(mysqlConf); + } + + @Test + public void testGetConfigMap() { + Map> config = new HashMap<>(); + config.put( + DatabaseSyncConfig.MYSQL_CONF, Arrays.asList(" hostname=127.0.0.1", " port=3306")); + config.put( + DatabaseSyncConfig.POSTGRES_CONF, + Arrays.asList("hostname=127.0.0.1 ", "port=5432 ")); + MultipleParameterTool parameter = MultipleParameterTool.fromMultiMap(config); + Map mysqlConfigMap = + CdcTools.getConfigMap(parameter, DatabaseSyncConfig.MYSQL_CONF); + Map postGresConfigMap = + CdcTools.getConfigMap(parameter, DatabaseSyncConfig.POSTGRES_CONF); + + Set mysqlKeyConf = new HashSet<>(Arrays.asList("hostname", "port")); + Set mysqlValueConf = new HashSet<>(Arrays.asList("127.0.0.1", "3306")); + assertEquals(mysqlConfigMap, mysqlKeyConf, mysqlValueConf); + + Set postgresKeyConf = new HashSet<>(Arrays.asList("hostname", "port")); + Set postgresValueConf = new HashSet<>(Arrays.asList("127.0.0.1", "5432")); + assertEquals(postGresConfigMap, postgresKeyConf, postgresValueConf); + } + + private void assertEquals( + Map actualMap, Set keyConf, Set valueConf) { + for (Entry entry : actualMap.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + Assert.assertTrue(keyConf.contains(key)); + Assert.assertTrue(valueConf.contains(value)); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java new file mode 100644 index 0000000..829fee5 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/MySqlContainer.java @@ -0,0 +1,174 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.oceanbase.cdc.tools.tests; + +import org.testcontainers.containers.ContainerLaunchException; +import org.testcontainers.containers.JdbcDatabaseContainer; +import org.testcontainers.utility.DockerImageName; + +import java.util.HashSet; +import java.util.Set; + +/** + * Docker container for MySQL. The difference between this class and {@link + * org.testcontainers.containers.MySqlContainer} is that TC MySQLContainer has problems when + * overriding mysql conf file, i.e. my.cnf. + */ +@SuppressWarnings("MagicNumber") +public class MySqlContainer extends JdbcDatabaseContainer { + + public static final String IMAGE = "mysql"; + public static final Integer MYSQL_PORT = 3306; + + private static final String MY_CNF_CONFIG_OVERRIDE_PARAM_NAME = "MY_CNF"; + private static final String SETUP_SQL_PARAM_NAME = "SETUP_SQL"; + private static final String MYSQL_ROOT_USER = "root"; + + private String databaseName = "test"; + private String username = "test"; + private String password = "test"; + + public MySqlContainer() { + super(DockerImageName.parse(IMAGE + ":" + "8.0.20")); + addExposedPort(MYSQL_PORT); + } + + @Override + protected Set getLivenessCheckPorts() { + return new HashSet<>(getMappedPort(MYSQL_PORT)); + } + + @Override + protected void configure() { + optionallyMapResourceParameterAsVolume( + MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, "/etc/mysql/", "mysql-default-conf"); + + if (parameters.containsKey(SETUP_SQL_PARAM_NAME)) { + optionallyMapResourceParameterAsVolume( + SETUP_SQL_PARAM_NAME, "/docker-entrypoint-initdb.d/", "N/A"); + } + + addEnv("MYSQL_DATABASE", databaseName); + addEnv("MYSQL_USER", username); + if (password != null && !password.isEmpty()) { + addEnv("MYSQL_PASSWORD", password); + addEnv("MYSQL_ROOT_PASSWORD", password); + } else { + throw new ContainerLaunchException( + "Empty password can be used only with the root user"); + } + setStartupAttempts(3); + } + + @Override + public String getDriverClassName() { + try { + Class.forName("com.mysql.cj.jdbc.Driver"); + return "com.mysql.cj.jdbc.Driver"; + } catch (ClassNotFoundException e) { + return "com.mysql.jdbc.Driver"; + } + } + + public String getJdbcUrl(String databaseName) { + String additionalUrlParams = constructUrlParameters("?", "&"); + return "jdbc:mysql://" + + getHost() + + ":" + + getDatabasePort() + + "/" + + databaseName + + additionalUrlParams; + } + + public void setDatabaseName(String databaseName) { + this.databaseName = databaseName; + } + + @Override + public String getJdbcUrl() { + return getJdbcUrl(databaseName); + } + + public int getDatabasePort() { + return getMappedPort(MYSQL_PORT); + } + + @Override + protected String constructUrlForConnection(String queryString) { + String url = super.constructUrlForConnection(queryString); + + if (!url.contains("useSSL=")) { + String separator = url.contains("?") ? "&" : "?"; + url = url + separator + "useSSL=false"; + } + + if (!url.contains("allowPublicKeyRetrieval=")) { + url = url + "&allowPublicKeyRetrieval=true"; + } + + return url; + } + + @Override + public String getDatabaseName() { + return databaseName; + } + + @Override + public String getUsername() { + return username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + protected String getTestQueryString() { + return "SELECT 1"; + } + + public MySqlContainer withConfigurationOverride(String s) { + parameters.put(MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, s); + return this; + } + + public MySqlContainer withSetupSQL(String sqlPath) { + parameters.put(SETUP_SQL_PARAM_NAME, sqlPath); + return this; + } + + @Override + public MySqlContainer withDatabaseName(final String databaseName) { + this.databaseName = databaseName; + return this; + } + + @Override + public MySqlContainer withUsername(final String username) { + this.username = username; + return this; + } + + @Override + public MySqlContainer withPassword(final String password) { + this.password = password; + return this; + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java new file mode 100644 index 0000000..4159c5a --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/java/com/oceanbase/cdc/tools/tests/OceanBaseJsonSerializationSchemaTest.java @@ -0,0 +1,186 @@ +/* + * Copyright 2024 OceanBase. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.oceanbase.cdc.tools.tests; + +import com.oceanbase.connector.flink.table.OceanBaseJsonSerializationSchema; +import com.oceanbase.connector.flink.table.SerializationRuntimeConverter; +import com.oceanbase.connector.flink.table.TableInfo; + +import org.apache.flink.table.data.*; +import org.apache.flink.table.types.logical.*; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.time.*; +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class OceanBaseJsonSerializationSchemaTest { + + @Test + void testCreateNotNullConverter() { + OceanBaseJsonSerializationSchemaConverterTest schema = + new OceanBaseJsonSerializationSchemaConverterTest(null); + // Test Boolean + assertEquals(true, schema.createNotNullConverter(new BooleanType()).convert(true)); + + // Test TinyInt + assertEquals( + (byte) 123, schema.createNotNullConverter(new TinyIntType()).convert((byte) 123)); + + // Test SmallInt + assertEquals( + (short) 12345, + schema.createNotNullConverter(new SmallIntType()).convert((short) 12345)); + + // Test Integer + assertEquals(123456789, schema.createNotNullConverter(new IntType()).convert(123456789)); + + // Test BigInt + assertEquals( + 123456789012345L, + schema.createNotNullConverter(new BigIntType()).convert(123456789012345L)); + + // Test Float + assertEquals(123.45f, schema.createNotNullConverter(new FloatType()).convert(123.45f)); + + // Test Double + assertEquals( + 123456.789, schema.createNotNullConverter(new DoubleType()).convert(123456.789)); + + // Test Binary + byte[] binaryData = "testBinary".getBytes(); + assertEquals( + binaryData, schema.createNotNullConverter(new BinaryType()).convert(binaryData)); + + // Test VarBinary + byte[] varBinaryData = "testVarBinary".getBytes(); + assertEquals( + varBinaryData, + schema.createNotNullConverter(new VarBinaryType(50)).convert(varBinaryData)); + + // Test Char + assertEquals("testChar", schema.createNotNullConverter(new CharType()).convert("testChar")); + + // Test Varchar + assertEquals( + "testVarchar", + schema.createNotNullConverter(new VarCharType()).convert("testVarchar")); + + LocalDate testDate = LocalDate.of(2024, 1, 1); + int daysSinceEpoch = (int) testDate.toEpochDay(); + Date expectedDate = Date.valueOf(testDate); + assertEquals( + expectedDate, + schema.createNotNullConverter(new DateType()).convert(daysSinceEpoch)); + + // Test TimeWithoutTimeZone + Time time = Time.valueOf("12:34:56"); + assertEquals( + time, + schema.createNotNullConverter(new TimeType()) + .convert(45296000)); // 45296000 ms equals 12:34:56 + + // Test TimestampWithoutTimeZone + TimestampData timestampData = + TimestampData.fromLocalDateTime(LocalDateTime.of(2024, 1, 1, 12, 34, 56)); + assertEquals( + timestampData.toTimestamp(), + schema.createNotNullConverter(new TimestampType()).convert(timestampData)); + + // Test TimestampWithTimeZone + TimestampData timestampWithTZ = + TimestampData.fromInstant(Instant.parse("2024-01-01T12:34:56Z")); + assertEquals( + "2024-01-01T12:34:56Z", + schema.createNotNullConverter(new ZonedTimestampType()).convert(timestampWithTZ)); + + // Test TimestampWithLocalTimeZone + TimestampData timestampWithLocalTZ = + TimestampData.fromInstant(Instant.parse("2024-01-01T12:34:56Z")); + assertEquals( + "2024-01-01T20:34:56+08:00[Asia/Shanghai]", + schema.createNotNullConverter(new LocalZonedTimestampType()) + .convert(timestampWithLocalTZ)); + + // Test Decimal + DecimalData decimalData = DecimalData.fromBigDecimal(new BigDecimal("123456.789"), 9, 3); + assertEquals( + new BigDecimal("123456.789"), + schema.createNotNullConverter(new DecimalType(9, 3)).convert(decimalData)); + + // Test Array + ArrayData arrayData = + new GenericArrayData( + new StringData[] { + StringData.fromString("a"), + StringData.fromString("b"), + StringData.fromString("c") + }); + String expectedArrayString = "a,b,c"; + String actualArrayString = + (String) + schema.createNotNullConverter(new ArrayType(new VarCharType())) + .convert(arrayData); + assertEquals(expectedArrayString, actualArrayString); + + // Test Map + Map map = new LinkedHashMap<>(); + map.put(StringData.fromString("key1"), StringData.fromString("value1")); + map.put(StringData.fromString("key2"), StringData.fromString("value2")); + MapData mapData = new GenericMapData(map); + // Assume there is a toString method that formats MapData correctly, or you can compare each + // element + String expectedMapString = "{\"key1\":\"value1\",\"key2\":\"value2\"}"; + String actualMapString = + (String) + schema.createNotNullConverter( + new MapType(new VarCharType(), new VarCharType())) + .convert(mapData); + assertEquals(expectedMapString, actualMapString); + + // Test Row + RowType rowType = + new RowType( + java.util.Arrays.asList( + new RowType.RowField("field1", new VarCharType()), + new RowType.RowField("field2", new IntType()), + new RowType.RowField("field3", new BooleanType()))); + RowData rowData = GenericRowData.of(StringData.fromString("field1"), 123, true); + assertEquals( + "{\"field1\":\"field1\",\"field2\":123,\"field3\":true}", + schema.createNotNullConverter(rowType).convert(rowData)); + } + + // Create a subclass to expose the protected createNotNullConverter method + private static class OceanBaseJsonSerializationSchemaConverterTest + extends OceanBaseJsonSerializationSchema { + + public OceanBaseJsonSerializationSchemaConverterTest(TableInfo tableInfo) { + super(tableInfo); + } + + @Override + public SerializationRuntimeConverter createNotNullConverter(LogicalType type) { + return super.createNotNullConverter(type); + } + } +} diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf b/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 0000000..a390897 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/docker/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties b/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000..68c8b84 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/log4j2-test.properties @@ -0,0 +1,21 @@ +# Copyright 2024 OceanBase. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +rootLogger.level = INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql b/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql new file mode 100644 index 0000000..dfb1378 --- /dev/null +++ b/flink-connector-oceanbase-tools-cdc/src/test/resources/sql/cdc.sql @@ -0,0 +1,11 @@ + +CREATE TABLE test_history_text ( + itemid bigint NOT NULL, + clock integer DEFAULT '0' NOT NULL, + value text NOT NULL, + ns integer DEFAULT '0' NOT NULL, + PRIMARY KEY (itemid,clock,ns) +); +INSERT INTO test_history_text (itemid,clock,value,ns) VALUES + (1,21131,'ces1',21321), + (2,21321,'ces2',12321); diff --git a/pom.xml b/pom.xml index 4a432c8..2efcc20 100644 --- a/pom.xml +++ b/pom.xml @@ -37,12 +37,16 @@ under the License. flink-sql-connector-oceanbase flink-sql-connector-oceanbase-directload flink-connector-oceanbase-e2e-tests + flink-connector-oceanbase-tools-cdc 1.5-SNAPSHOT 1.18.0 2.12 + 19.3.0.0 + 3.1.1 + 1.17.6 com.oceanbase.connector.flink.shaded