From 6b994afa591580b80e6de6c930e52d9fe2083f97 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 17 May 2024 09:23:41 +0800 Subject: [PATCH 01/23] newFeature: init oceanbase flink cdc pipeline connector --- .../pom.xml | 65 +++ .../oceanbase/catalog/OceanBaseCatalog.java | 62 +++ .../catalog/OceanBaseCatalogException.java | 38 ++ .../catalog/OceanBaseCatalogFactory.java | 25 ++ .../oceanbase/catalog/OceanBaseColumn.java | 227 +++++++++++ .../catalog/OceanBaseMySQLCatalog.java | 303 ++++++++++++++ .../catalog/OceanBaseOracleCatalog.java | 264 ++++++++++++ .../oceanbase/catalog/OceanBaseTable.java | 278 +++++++++++++ .../factory/OceanBaseDataSinkFactory.java | 86 ++++ .../oceanbase/sink/MySQLDialect.java | 21 + .../oceanbase/sink/OceanBaseDataSink.java | 79 ++++ .../sink/OceanBaseDataSinkOptions.java | 163 ++++++++ .../sink/OceanBaseEventSerializer.java | 120 ++++++ .../sink/OceanBaseMetadataApplier.java | 144 +++++++ .../oceanbase/sink/OceanBaseUtils.java | 382 ++++++++++++++++++ ....apache.flink.cdc.common.factories.Factory | 16 + .../oceanbase/OceanBaseContainer.java | 130 ++++++ .../oceanbase/OceanBaseTestBase.java | 50 +++ .../catalog/OceanBaseMySQLCatalogTest.java | 36 ++ .../src/test/resources/sql/init.sql | 39 ++ .../flink-cdc-pipeline-connectors/pom.xml | 1 + 21 files changed, 2529 insertions(+) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/resources/sql/init.sql diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml new file mode 100644 index 00000000000..46969495313 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -0,0 +1,65 @@ + + + + 4.0.0 + + org.apache.flink + flink-cdc-pipeline-connectors + 3.2-SNAPSHOT + + + flink-cdc-pipeline-connector-oceanbase + + + + + + + com.oceanbase + flink-connector-oceanbase + 1.1 + + + com.google.guava + guava + + + + + com.oceanbase + oceanbase-client + 2.4.8 + test + + + + + + + + + + org.testcontainers + jdbc + ${testcontainers.version} + test + + + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java new file mode 100644 index 00000000000..d67a06a693a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java @@ -0,0 +1,62 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; +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.List; +import java.util.Objects; + +/** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution. */ +public abstract class OceanBaseCatalog { + + private final OceanBaseConnectionProvider connectionProvider; + + public OceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) { + assert Objects.nonNull(connectorOptions); + this.connectionProvider = new OceanBaseConnectionProvider(connectorOptions); + } + + protected List executeSingleColumnStatement(String sql) throws SQLException { + try (Connection conn = connectionProvider.getConnection(); + PreparedStatement statement = conn.prepareStatement(sql)) { + List columnValues = Lists.newArrayList(); + try (ResultSet rs = statement.executeQuery()) { + while (rs.next()) { + String columnValue = rs.getString(1); + columnValues.add(columnValue); + } + } + return columnValues; + } + } + + protected void executeUpdateStatement(String sql) throws SQLException { + try (Connection connection = connectionProvider.getConnection(); + Statement statement = connection.createStatement()) { + statement.executeUpdate(sql); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java new file mode 100644 index 00000000000..ed12088f465 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java @@ -0,0 +1,38 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +public class OceanBaseCatalogException extends RuntimeException { + /** @param message the detail message. */ + public OceanBaseCatalogException(String message) { + super(message); + } + + /** @param cause the cause. */ + public OceanBaseCatalogException(Throwable cause) { + super(cause); + } + + /** + * @param message the detail message. + * @param cause the cause. + */ + public OceanBaseCatalogException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java new file mode 100644 index 00000000000..45ff4f52afa --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java @@ -0,0 +1,25 @@ +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; +import com.oceanbase.connector.flink.dialect.OceanBaseDialect; +import com.oceanbase.connector.flink.dialect.OceanBaseMySQLDialect; +import com.oceanbase.connector.flink.dialect.OceanBaseOracleDialect; +import org.apache.flink.cdc.common.factories.DataSinkFactory; + +/** A {@link OceanBaseCatalogFactory} to create {@link OceanBaseCatalog}. */ +public class OceanBaseCatalogFactory { + + public static OceanBaseCatalog createOceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) throws Exception { + try (OceanBaseConnectionProvider connectionProvider = new OceanBaseConnectionProvider(connectorOptions)) { + OceanBaseDialect dialect = connectionProvider.getDialect(); + if (dialect instanceof OceanBaseMySQLDialect) { + return new OceanBaseMySQLCatalog(connectorOptions); + } else if (dialect instanceof OceanBaseOracleDialect) { + return new OceanBaseOracleCatalog(connectorOptions); + }else { + throw new OceanBaseCatalogException("This tenant is not supported currently"); + } + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java new file mode 100644 index 00000000000..1d4f501a9c8 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java @@ -0,0 +1,227 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Describe a column of OceanBase table. These metas are from information_schema.COLUMNS. */ +public class OceanBaseColumn implements Serializable { + + private static final long serialVersionUID = 1L; + + /** The name of the column. COLUMN_NAME in information_schema.COLUMNS. */ + private final String columnName; + + /** + * The position of the column within the table (starting at 0). ORDINAL_POSITION in + * information_schema.COLUMNS. + */ + private final int ordinalPosition; + + /** The column data type. DATA_TYPE in information_schema.COLUMNS. */ + private final String dataType; + + /** The column nullability.IS_NULLABLE in information_schema.COLUMNS. */ + private final boolean isNullable; + + /** The default value for the column. COLUMN_DEFAULT in information_schema.COLUMNS. */ + @Nullable private final String defaultValue; + + /** + * The column size. COLUMN_SIZE in information_schema.COLUMNS. For numeric data, this is the + * maximum precision. For character data, this is the length in characters. For other data + * types, this is null. + */ + @Nullable private final Integer columnSize; + + /** + * The number of fractional digits for numeric data. This is null for other data types. + * DECIMAL_DIGITS in information_schema.COLUMNS. + */ + @Nullable private final Integer decimalDigits; + + /** The column comment. COLUMN_COMMENT in information_schema.COLUMNS. */ + @Nullable private final String columnComment; + + private OceanBaseColumn( + String columnName, + int ordinalPosition, + String dataType, + boolean isNullable, + @Nullable String defaultValue, + @Nullable Integer columnSize, + @Nullable Integer decimalDigits, + @Nullable String columnComment) { + this.columnName = checkNotNull(columnName); + this.ordinalPosition = ordinalPosition; + this.dataType = checkNotNull(dataType); + this.isNullable = isNullable; + this.defaultValue = defaultValue; + this.columnSize = columnSize; + this.decimalDigits = decimalDigits; + this.columnComment = columnComment; + } + + public String getColumnName() { + return columnName; + } + + public int getOrdinalPosition() { + return ordinalPosition; + } + + public String getDataType() { + return dataType; + } + + public boolean isNullable() { + return isNullable; + } + + public Optional getDefaultValue() { + return Optional.ofNullable(defaultValue); + } + + public Optional getColumnSize() { + return Optional.ofNullable(columnSize); + } + + public Optional getDecimalDigits() { + return Optional.ofNullable(decimalDigits); + } + + public Optional getColumnComment() { + return Optional.ofNullable(columnComment); + } + + @Override + public String toString() { + return "OceanBaseColumn{" + + "columnName='" + + columnName + + '\'' + + ", ordinalPosition=" + + ordinalPosition + + ", dataType='" + + dataType + + '\'' + + ", isNullable=" + + isNullable + + ", defaultValue='" + + defaultValue + + '\'' + + ", columnSize=" + + columnSize + + ", decimalDigits=" + + decimalDigits + + ", columnComment='" + + columnComment + + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OceanBaseColumn column = (OceanBaseColumn) o; + return ordinalPosition == column.ordinalPosition + && isNullable == column.isNullable + && Objects.equals(columnName, column.columnName) + && dataType.equalsIgnoreCase(column.dataType) + && Objects.equals(defaultValue, column.defaultValue) + && Objects.equals(columnSize, column.columnSize) + && Objects.equals(decimalDigits, column.decimalDigits) + && Objects.equals(columnComment, column.columnComment); + } + + /** Build a {@link OceanBaseColumn}. */ + public static class Builder { + + private String columnName; + private int ordinalPosition; + private String dataType; + private boolean isNullable = true; + private String defaultValue; + private Integer columnSize; + private Integer decimalDigits; + private String columnComment; + + public OceanBaseColumn.Builder setColumnName(String columnName) { + this.columnName = columnName; + return this; + } + + public OceanBaseColumn.Builder setOrdinalPosition(int ordinalPosition) { + this.ordinalPosition = ordinalPosition; + return this; + } + + public OceanBaseColumn.Builder setDataType(String dataType) { + this.dataType = dataType; + return this; + } + + public OceanBaseColumn.Builder setNullable(boolean isNullable) { + this.isNullable = isNullable; + return this; + } + + public OceanBaseColumn.Builder setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + return this; + } + + public OceanBaseColumn.Builder setColumnSize(Integer columnSize) { + this.columnSize = columnSize; + return this; + } + + public OceanBaseColumn.Builder setDecimalDigits(Integer decimalDigits) { + this.decimalDigits = decimalDigits; + return this; + } + + public OceanBaseColumn.Builder setColumnComment(String columnComment) { + this.columnComment = columnComment; + return this; + } + + public OceanBaseColumn build() { + return new OceanBaseColumn( + columnName, + ordinalPosition, + dataType, + isNullable, + defaultValue, + columnSize, + decimalDigits, + columnComment); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java new file mode 100644 index 00000000000..53b56ec45df --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -0,0 +1,303 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under MySQL mode. */ +public class OceanBaseMySQLCatalog extends OceanBaseCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLCatalog.class); + + public OceanBaseMySQLCatalog(OceanBaseConnectorOptions connectorOptions) { + super(connectorOptions); + } + + /** + * Check if a database exists in this catalog. + * + * @param databaseName Name of the database + * @return true if the given database exists in the catalog false otherwise + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public boolean databaseExists(String databaseName) throws OceanBaseCatalogException { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + String querySql = + String.format( + "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA` WHERE SCHEMA_NAME = '%s';", + databaseName); + try { + List dbList = executeSingleColumnStatement(querySql); + return !dbList.isEmpty(); + } catch (Exception e) { + LOG.error( + "Failed to check database exist, database: {}, sql: {}", + databaseName, + querySql, + e); + throw new OceanBaseCatalogException( + String.format("Failed to check database exist, database: %s", databaseName), e); + } + } + + /** + * Create a database. + * + * @param databaseName Name of the database + * @param ignoreIfExists Flag to specify behavior when a database with the given name already + * exists: if set to false, throw a StarRocksCatalogException, if set to true, do nothing. + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public void createDatabase(String databaseName, boolean ignoreIfExists) + throws OceanBaseCatalogException { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + String sql = buildCreateDatabaseSql(databaseName, ignoreIfExists); + try { + executeUpdateStatement(sql); + LOG.info("Successful to create database {}, sql: {}", databaseName, sql); + } catch (Exception e) { + LOG.info("Failed to create database {}, sql: {}", databaseName, sql, e); + throw new OceanBaseCatalogException( + String.format( + "Failed to create database %s, ignoreIfExists: %s", + databaseName, ignoreIfExists), + e); + } + } + + /** + * Creates a table. + * + * @param table the table definition + * @param ignoreIfExists flag to specify behavior when a table already exists. if set to false, + * it throws a TableAlreadyExistException, if set to true, do nothing. + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public void createTable(OceanBaseTable table, boolean ignoreIfExists) + throws OceanBaseCatalogException { + String createTableSql = buildCreateTableSql(table, ignoreIfExists); + try { + executeUpdateStatement(createTableSql); + LOG.info( + "Success to create table {}.{}, sql: {}", + table.getDatabaseName(), + table.getDatabaseName(), + createTableSql); + } catch (Exception e) { + LOG.error( + "Failed to create table {}.{}, sql: {}", + table.getDatabaseName(), + table.getDatabaseName(), + createTableSql, + e); + throw new OceanBaseCatalogException( + String.format( + "Failed to create table %s.%s", + table.getDatabaseName(), table.getDatabaseName()), + e); + } + } + + public void alterAddColumns(String databaseName, String tableName, List addColumns) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + Preconditions.checkArgument(!addColumns.isEmpty(), "Added columns should not be empty."); + + String alterSql = + buildAlterAddColumnsSql(databaseName, tableName, addColumns); + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(alterSql); + LOG.info("Success to add columns to {}.{}, duration: {}ms, sql: {}", + databaseName, tableName, System.currentTimeMillis() - startTimeMillis, alterSql); + } catch (Exception e) { + LOG.error("Failed to add columns to {}.{}, sql: {}", databaseName, tableName, alterSql, e); + throw new OceanBaseCatalogException( + String.format("Failed to add columns to %s.%s ", databaseName, tableName), e); + } + } + + // ------------------------------------------------------------------------------------------ + // OceanBase DDL SQL + // ------------------------------------------------------------------------------------------ + + private String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + return String.format( + "CREATE DATABASE %s%s;", ignoreIfExists ? "IF NOT EXISTS " : "", databaseName); + } + + private String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { + StringBuilder builder = new StringBuilder(); + builder.append( + String.format( + "CREATE TABLE %s`%s`.`%s`", + ignoreIfExists ? "IF NOT EXISTS " : "", + table.getDatabaseName(), + table.getTableName())); + builder.append(" (\n"); + String columnsStmt = + table.getColumns().stream() + .map(this::buildColumnStmt) + .collect(Collectors.joining(",\n")); + builder.append(columnsStmt); + builder.append("\n) "); + + Preconditions.checkArgument( + table.getTableType() == OceanBaseTable.TableType.PRIMARY_KEY, + "Not support to build create table sql for table type " + table.getTableType()); + Preconditions.checkArgument( + table.getTableKeys().isPresent(), + "Can't build create table sql because there is no table keys"); + String tableKeys = + table.getTableKeys().get().stream() + .map(key -> "`" + key + "`") + .collect(Collectors.joining(", ")); + builder.append(String.format("PRIMARY KEY (%s)\n", tableKeys)); + + Preconditions.checkArgument( + table.getDistributionKeys().isPresent(), + "Can't build create table sql because there is no distribution keys"); + String distributionKeys = + table.getDistributionKeys().get().stream() + .map(key -> "`" + key + "`") + .collect(Collectors.joining(", ")); + builder.append(String.format("DISTRIBUTED BY HASH (%s)", distributionKeys)); + if (table.getNumBuckets().isPresent()) { + builder.append(" BUCKETS "); + builder.append(table.getNumBuckets().get()); + } + if (!table.getProperties().isEmpty()) { + builder.append("\nPROPERTIES (\n"); + String properties = + table.getProperties().entrySet().stream() + .map( + entry -> + String.format( + "\"%s\" = \"%s\"", + entry.getKey(), entry.getValue())) + .collect(Collectors.joining(",\n")); + builder.append(properties); + builder.append("\n)"); + } + builder.append(";"); + return builder.toString(); + } + + private String buildAlterAddColumnsSql( + String databaseName, + String tableName, + List addColumns, + long timeoutSecond) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + addColumns.stream() + .map(col -> "ADD COLUMN " + buildColumnStmt(col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); + builder.append(";"); + return builder.toString(); + } + + private String buildAlterDropColumnsSql( + String databaseName, String tableName, List dropColumns, long timeoutSecond) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + dropColumns.stream() + .map(col -> String.format("DROP COLUMN `%s`", col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); + builder.append(";"); + return builder.toString(); + } + + private String buildColumnStmt(OceanBaseColumn column) { + StringBuilder builder = new StringBuilder(); + builder.append("`"); + builder.append(column.getColumnName()); + builder.append("` "); + builder.append( + getFullColumnType( + column.getDataType(), column.getColumnSize(), column.getDecimalDigits())); + builder.append(" "); + builder.append(column.isNullable() ? "NULL" : "NOT NULL"); + if (column.getDefaultValue().isPresent()) { + builder.append(String.format(" DEFAULT \"%s\"", column.getDefaultValue().get())); + } + + if (column.getColumnComment().isPresent()) { + builder.append(String.format(" COMMENT \"%s\"", column.getColumnComment().get())); + } + return builder.toString(); + } + + private String getFullColumnType( + String type, Optional columnSize, Optional decimalDigits) { + String dataType = type.toUpperCase(); + switch (dataType) { + case "DECIMAL": + Preconditions.checkArgument( + columnSize.isPresent(), "DECIMAL type must have column size"); + Preconditions.checkArgument( + decimalDigits.isPresent(), "DECIMAL type must have decimal digits"); + return String.format("DECIMAL(%d, %s)", columnSize.get(), decimalDigits.get()); + case "CHAR": + case "VARCHAR": + Preconditions.checkArgument( + columnSize.isPresent(), type + " type must have column size"); + return String.format("%s(%d)", dataType, columnSize.get()); + default: + return dataType; + } + } + + protected String buildAlterAddColumnsSql( + String databaseName, + String tableName, + List addColumns) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + addColumns.stream() + .map(col -> "ADD COLUMN " + buildColumnStmt(col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(";"); + return builder.toString(); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java new file mode 100644 index 00000000000..76f3cf73b34 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java @@ -0,0 +1,264 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under Oracle mode. */ +public class OceanBaseOracleCatalog extends OceanBaseCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseOracleCatalog.class); + + public OceanBaseOracleCatalog(OceanBaseConnectorOptions connectorOptions) { + super(connectorOptions); + } + + /** + * Check if a database exists in this catalog. + * + * @param databaseName Name of the database + * @return true if the given database exists in the catalog false otherwise + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public boolean databaseExists(String databaseName) throws OceanBaseCatalogException { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + String querySql = + String.format( + "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA` WHERE SCHEMA_NAME = '%s';", + databaseName); + try { + List dbList = executeSingleColumnStatement(querySql); + return !dbList.isEmpty(); + } catch (Exception e) { + LOG.error( + "Failed to check database exist, database: {}, sql: {}", + databaseName, + querySql, + e); + throw new OceanBaseCatalogException( + String.format("Failed to check database exist, database: %s", databaseName), e); + } + } + + /** + * Create a database. + * + * @param databaseName Name of the database + * @param ignoreIfExists Flag to specify behavior when a database with the given name already + * exists: if set to false, throw a StarRocksCatalogException, if set to true, do nothing. + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public void createDatabase(String databaseName, boolean ignoreIfExists) + throws OceanBaseCatalogException { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + String sql = buildCreateDatabaseSql(databaseName, ignoreIfExists); + try { + executeUpdateStatement(sql); + LOG.info("Successful to create database {}, sql: {}", databaseName, sql); + } catch (Exception e) { + LOG.info("Failed to create database {}, sql: {}", databaseName, sql, e); + throw new OceanBaseCatalogException( + String.format( + "Failed to create database %s, ignoreIfExists: %s", + databaseName, ignoreIfExists), + e); + } + } + + /** + * Creates a table. + * + * @param table the table definition + * @param ignoreIfExists flag to specify behavior when a table already exists. if set to false, + * it throws a TableAlreadyExistException, if set to true, do nothing. + * @throws OceanBaseCatalogException in case of any runtime exception + */ + public void createTable(OceanBaseTable table, boolean ignoreIfExists) + throws OceanBaseCatalogException { + String createTableSql = buildCreateTableSql(table, ignoreIfExists); + try { + executeUpdateStatement(createTableSql); + LOG.info( + "Success to create table {}.{}, sql: {}", + table.getDatabaseName(), + table.getDatabaseName(), + createTableSql); + } catch (Exception e) { + LOG.error( + "Failed to create table {}.{}, sql: {}", + table.getDatabaseName(), + table.getDatabaseName(), + createTableSql, + e); + throw new OceanBaseCatalogException( + String.format( + "Failed to create table %s.%s", + table.getDatabaseName(), table.getDatabaseName()), + e); + } + } + + // ------------------------------------------------------------------------------------------ + // OceanBase DDL SQL + // ------------------------------------------------------------------------------------------ + + private String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + return String.format( + "CREATE DATABASE %s%s;", ignoreIfExists ? "IF NOT EXISTS " : "", databaseName); + } + + private String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { + StringBuilder builder = new StringBuilder(); + builder.append( + String.format( + "CREATE TABLE %s`%s`.`%s`", + ignoreIfExists ? "IF NOT EXISTS " : "", + table.getDatabaseName(), + table.getTableName())); + builder.append(" (\n"); + String columnsStmt = + table.getColumns().stream() + .map(this::buildColumnStmt) + .collect(Collectors.joining(",\n")); + builder.append(columnsStmt); + builder.append("\n) "); + + Preconditions.checkArgument( + table.getTableType() == OceanBaseTable.TableType.PRIMARY_KEY, + "Not support to build create table sql for table type " + table.getTableType()); + Preconditions.checkArgument( + table.getTableKeys().isPresent(), + "Can't build create table sql because there is no table keys"); + String tableKeys = + table.getTableKeys().get().stream() + .map(key -> "`" + key + "`") + .collect(Collectors.joining(", ")); + builder.append(String.format("PRIMARY KEY (%s)\n", tableKeys)); + + Preconditions.checkArgument( + table.getDistributionKeys().isPresent(), + "Can't build create table sql because there is no distribution keys"); + String distributionKeys = + table.getDistributionKeys().get().stream() + .map(key -> "`" + key + "`") + .collect(Collectors.joining(", ")); + builder.append(String.format("DISTRIBUTED BY HASH (%s)", distributionKeys)); + if (table.getNumBuckets().isPresent()) { + builder.append(" BUCKETS "); + builder.append(table.getNumBuckets().get()); + } + if (!table.getProperties().isEmpty()) { + builder.append("\nPROPERTIES (\n"); + String properties = + table.getProperties().entrySet().stream() + .map( + entry -> + String.format( + "\"%s\" = \"%s\"", + entry.getKey(), entry.getValue())) + .collect(Collectors.joining(",\n")); + builder.append(properties); + builder.append("\n)"); + } + builder.append(";"); + return builder.toString(); + } + + private String buildAlterAddColumnsSql( + String databaseName, + String tableName, + List addColumns, + long timeoutSecond) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + addColumns.stream() + .map(col -> "ADD COLUMN " + buildColumnStmt(col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); + builder.append(";"); + return builder.toString(); + } + + private String buildAlterDropColumnsSql( + String databaseName, String tableName, List dropColumns, long timeoutSecond) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + dropColumns.stream() + .map(col -> String.format("DROP COLUMN `%s`", col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); + builder.append(";"); + return builder.toString(); + } + + private String buildColumnStmt(OceanBaseColumn column) { + StringBuilder builder = new StringBuilder(); + builder.append("`"); + builder.append(column.getColumnName()); + builder.append("` "); + builder.append( + getFullColumnType( + column.getDataType(), column.getColumnSize(), column.getDecimalDigits())); + builder.append(" "); + builder.append(column.isNullable() ? "NULL" : "NOT NULL"); + if (column.getDefaultValue().isPresent()) { + builder.append(String.format(" DEFAULT \"%s\"", column.getDefaultValue().get())); + } + + if (column.getColumnComment().isPresent()) { + builder.append(String.format(" COMMENT \"%s\"", column.getColumnComment().get())); + } + return builder.toString(); + } + + private String getFullColumnType( + String type, Optional columnSize, Optional decimalDigits) { + String dataType = type.toUpperCase(); + switch (dataType) { + case "DECIMAL": + Preconditions.checkArgument( + columnSize.isPresent(), "DECIMAL type must have column size"); + Preconditions.checkArgument( + decimalDigits.isPresent(), "DECIMAL type must have decimal digits"); + return String.format("DECIMAL(%d, %s)", columnSize.get(), decimalDigits.get()); + case "CHAR": + case "VARCHAR": + Preconditions.checkArgument( + columnSize.isPresent(), type + " type must have column size"); + return String.format("%s(%d)", dataType, columnSize.get()); + default: + return dataType; + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java new file mode 100644 index 00000000000..4cf2f39a320 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java @@ -0,0 +1,278 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** + * Describe a OceanBase table. See OceanBase table + * design for how to define a OceanBase table. + */ +public class OceanBaseTable { + + /** + * Types of OceanBase table. See OceanBase Table Types. + */ + public enum TableType { + UNKNOWN, + DUPLICATE_KEY, + AGGREGATE, + UNIQUE_KEY, + PRIMARY_KEY + } + + /** The database name. */ + private final String databaseName; + + /** The table name. */ + private final String tableName; + + /** The type of OceanBase type. */ + private final TableType tableType; + + /** The columns sorted by the ordinal position. */ + private final List columns; + + /** + * The table keys sorted by the ordinal position. null if it's unknown. The table keys has + * different meaning for different types of tables. For duplicate key table, It's duplicate + * keys. For aggregate table, it's aggregate keys. For unique key table, it's unique keys. For + * primary key table, it's primary keys. + */ + @Nullable private final List tableKeys; + + /** The distribution keys. null if it's unknown. */ + @Nullable private final List distributionKeys; + + /** The number of buckets. null if it's unknown or automatic. */ + @Nullable private final Integer numBuckets; + + /** The table comment. null if there is no comment or it's unknown. */ + @Nullable private final String comment; + + /** The properties of the table. */ + private final Map properties; + + /** Map the column name to the column. May be lazily initialized. */ + @Nullable private volatile Map columnMap; + + private OceanBaseTable( + String databaseName, + String tableName, + TableType tableType, + List columns, + @Nullable List tableKeys, + @Nullable List distributionKeys, + @Nullable Integer numBuckets, + @Nullable String comment, + Map properties) { + Preconditions.checkNotNull(databaseName); + Preconditions.checkNotNull(tableName); + Preconditions.checkNotNull(tableType); + Preconditions.checkArgument(columns != null && !columns.isEmpty()); + this.databaseName = databaseName; + this.tableName = tableName; + this.tableType = tableType; + this.columns = columns; + this.tableKeys = tableKeys; + this.distributionKeys = distributionKeys; + this.numBuckets = numBuckets; + this.comment = comment; + this.properties = Preconditions.checkNotNull(properties); + } + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public TableType getTableType() { + return tableType; + } + + public List getColumns() { + return columns; + } + + public Optional> getTableKeys() { + return Optional.ofNullable(tableKeys); + } + + public Optional> getDistributionKeys() { + return Optional.ofNullable(distributionKeys); + } + + public Optional getNumBuckets() { + return Optional.ofNullable(numBuckets); + } + + public Optional getComment() { + return Optional.ofNullable(comment); + } + + public Map getProperties() { + return properties; + } + + public OceanBaseColumn getColumn(String columnName) { + if (columnMap == null) { + synchronized (this) { + if (columnMap == null) { + columnMap = new HashMap<>(); + for (OceanBaseColumn column : columns) { + columnMap.put(column.getColumnName(), column); + } + } + } + } + return columnMap.get(columnName); + } + + @Override + public String toString() { + return "OceanBaseTable{" + + "databaseName='" + + databaseName + + '\'' + + ", tableName='" + + tableName + + '\'' + + ", tableType=" + + tableType + + ", columns=" + + columns + + ", tableKeys=" + + tableKeys + + ", distributionKeys=" + + distributionKeys + + ", numBuckets=" + + numBuckets + + ", comment='" + + comment + + '\'' + + ", properties=" + + properties + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OceanBaseTable that = (OceanBaseTable) o; + return Objects.equals(databaseName, that.databaseName) + && Objects.equals(tableName, that.tableName) + && tableType == that.tableType + && Objects.equals(columns, that.columns) + && Objects.equals(tableKeys, that.tableKeys) + && Objects.equals(distributionKeys, that.distributionKeys) + && Objects.equals(numBuckets, that.numBuckets) + && Objects.equals(comment, that.comment) + && Objects.equals(properties, that.properties); + } + + /** Build a {@link OceanBaseTable}. */ + public static class Builder { + + private String databaseName; + private String tableName; + private TableType tableType; + private List columns = new ArrayList<>(); + private List tableKeys; + private List distributionKeys; + private Integer numBuckets; + private String comment; + private Map properties = new HashMap<>(); + + public Builder setDatabaseName(String databaseName) { + this.databaseName = databaseName; + return this; + } + + public Builder setTableName(String tableName) { + this.tableName = tableName; + return this; + } + + public Builder setTableType(TableType tableType) { + this.tableType = tableType; + return this; + } + + public Builder setColumns(List columns) { + this.columns = columns; + return this; + } + + public Builder setTableKeys(List tableKeys) { + this.tableKeys = tableKeys; + return this; + } + + public Builder setDistributionKeys(List distributionKeys) { + this.distributionKeys = distributionKeys; + return this; + } + + public Builder setNumBuckets(Integer numBuckets) { + this.numBuckets = numBuckets; + return this; + } + + public Builder setComment(String comment) { + this.comment = comment; + return this; + } + + public Builder setTableProperties(Map properties) { + this.properties = properties; + return this; + } + + public OceanBaseTable build() { + return new OceanBaseTable( + databaseName, + tableName, + tableType, + columns, + tableKeys, + distributionKeys, + numBuckets, + comment, + properties); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java new file mode 100644 index 00000000000..3ab247dfa49 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -0,0 +1,86 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.factory; + +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.factories.DataSinkFactory; +import org.apache.flink.cdc.common.pipeline.PipelineOptions; +import org.apache.flink.cdc.common.sink.DataSink; +import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSink; +import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSinkOptions; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; + +import java.time.ZoneId; +import java.util.HashSet; +import java.util.Set; + +public class OceanBaseDataSinkFactory implements DataSinkFactory { + + @Override + public DataSink createDataSink(Context context) { + Configuration config = context.getFactoryConfiguration(); + OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(config.toMap()); + + return new OceanBaseDataSink( + connectorOptions, + config, + ZoneId.of( + context.getPipelineConfiguration() + .get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE))); + } + + @Override + public String identifier() { + return "oceanbase"; + } + + @Override + public Set> requiredOptions() { + Set> requiredOptions = new HashSet<>(); + requiredOptions.add(OceanBaseDataSinkOptions.URL); + requiredOptions.add(OceanBaseDataSinkOptions.USERNAME); + requiredOptions.add(OceanBaseDataSinkOptions.PASSWORD); + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + Set> optionalOptions = new HashSet<>(); + optionalOptions.add(OceanBaseDataSinkOptions.DRIVER_CLASS_NAME); + optionalOptions.add(OceanBaseDataSinkOptions.DRUID_PROPERTIES); + optionalOptions.add(OceanBaseDataSinkOptions.MEMSTORE_CHECK_ENABLED); + optionalOptions.add(OceanBaseDataSinkOptions.MEMSTORE_THRESHOLD); + optionalOptions.add(OceanBaseDataSinkOptions.MEMSTORE_CHECK_INTERVAL); + optionalOptions.add(OceanBaseDataSinkOptions.PARTITION_ENABLED); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_ENABLED); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_HOST); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_PORT); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_PARALLEL); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_MAX_ERROR_ROWS); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_DUP_ACTION); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_TIMEOUT); + optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_HEARTBEAT_TIMEOUT); + optionalOptions.add(OceanBaseDataSinkOptions.SYNC_WRITE); + optionalOptions.add(OceanBaseDataSinkOptions.BUFFER_FLUSH_INTERVAL); + optionalOptions.add(OceanBaseDataSinkOptions.BUFFER_SIZE); + optionalOptions.add(OceanBaseDataSinkOptions.MAX_RETRIES); + return optionalOptions; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java new file mode 100644 index 00000000000..834aba557b7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java @@ -0,0 +1,21 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +/*MySQL Dialect*/ +public class MySQLDialect {} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java new file mode 100644 index 00000000000..890f952c43a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -0,0 +1,79 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.sink.DataSink; +import org.apache.flink.cdc.common.sink.EventSinkProvider; +import org.apache.flink.cdc.common.sink.FlinkSinkProvider; +import org.apache.flink.cdc.common.sink.MetadataApplier; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; +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.TableId; + +import java.io.Serializable; +import java.time.ZoneId; + +public class OceanBaseDataSink implements DataSink, Serializable { + + private final OceanBaseConnectorOptions connectorOptions; + + private final Configuration config; + + private final ZoneId zoneId; + + public OceanBaseDataSink( + OceanBaseConnectorOptions options, Configuration config, ZoneId zoneId) { + this.connectorOptions = options; + this.config = config; + this.zoneId = zoneId; + } + + @Override + public EventSinkProvider getEventSinkProvider() { + OceanBaseConnectionProvider connectionProvider = + new OceanBaseConnectionProvider(connectorOptions); + TableId tableId = + new TableId( + connectionProvider.getDialect()::getFullTableName, + connectorOptions.getSchemaName(), + connectorOptions.getTableName()); + OceanBaseRecordFlusher recordFlusher = + new OceanBaseRecordFlusher(connectorOptions, connectionProvider); + return FlinkSinkProvider.of( + new OceanBaseSink<>( + connectorOptions, + null, + new OceanBaseEventSerializer(zoneId), + DataChangeRecord.KeyExtractor.simple(), + recordFlusher)); + } + + @Override + public MetadataApplier getMetadataApplier() { + try { + return new OceanBaseMetadataApplier(connectorOptions, config); + } catch (Exception e) { + throw new RuntimeException("Fail to create metadata applier"); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java new file mode 100644 index 00000000000..96dba6a53d5 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java @@ -0,0 +1,163 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.configuration.ConfigOptions; + +import com.alipay.oceanbase.rpc.protocol.payload.impl.direct_load.ObLoadDupActionType; + +import java.time.Duration; + +public class OceanBaseDataSinkOptions { + // ------------------------------------------------------------------------------------------ + // Options for sink connector + // ------------------------------------------------------------------------------------------ + public static final ConfigOption URL = + ConfigOptions.key("url") + .stringType() + .noDefaultValue() + .withDescription("The connection URL."); + + public static final ConfigOption USERNAME = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("The username."); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("The password."); + + public static final ConfigOption DRIVER_CLASS_NAME = + ConfigOptions.key("driver-class-name") + .stringType() + .defaultValue("com.mysql.cj.jdbc.Driver") + .withDescription( + "JDBC driver class name, use 'com.mysql.cj.jdbc.Driver' by default."); + + public static final ConfigOption DRUID_PROPERTIES = + ConfigOptions.key("druid-properties") + .stringType() + .noDefaultValue() + .withDescription("Properties for specific connection pool."); + + public static final ConfigOption MEMSTORE_CHECK_ENABLED = + ConfigOptions.key("memstore-check.enabled") + .booleanType() + .defaultValue(true) + .withDescription("Whether enable memstore check. Default value is 'true'"); + + public static final ConfigOption MEMSTORE_THRESHOLD = + ConfigOptions.key("memstore-check.threshold") + .doubleType() + .defaultValue(0.9) + .withDescription( + "Memory usage threshold ratio relative to the limit value. Default value is '0.9'."); + + public static final ConfigOption MEMSTORE_CHECK_INTERVAL = + ConfigOptions.key("memstore-check.interval") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "The check interval, over this time, the writer will check if memstore reaches threshold. Default value is '30s'."); + + public static final ConfigOption PARTITION_ENABLED = + ConfigOptions.key("partition.enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to enable partition calculation and flush records by partitions. Default value is 'false'."); + + public static final ConfigOption DIRECT_LOAD_ENABLED = + ConfigOptions.key("direct-load.enabled") + .booleanType() + .defaultValue(false) + .withDescription("Whether to enable direct load."); + + public static final ConfigOption DIRECT_LOAD_HOST = + ConfigOptions.key("direct-load.host") + .stringType() + .noDefaultValue() + .withDescription("Hostname used in direct load."); + + public static final ConfigOption DIRECT_LOAD_PORT = + ConfigOptions.key("direct-load.port") + .intType() + .defaultValue(2882) + .withDescription("Rpc port number used in direct load."); + + public static final ConfigOption DIRECT_LOAD_PARALLEL = + ConfigOptions.key("direct-load.parallel") + .intType() + .defaultValue(8) + .withDescription("Parallelism of direct load."); + + public static final ConfigOption DIRECT_LOAD_MAX_ERROR_ROWS = + ConfigOptions.key("direct-load.max-error-rows") + .longType() + .defaultValue(0L) + .withDescription("Maximum tolerable number of error rows."); + + public static final ConfigOption DIRECT_LOAD_DUP_ACTION = + ConfigOptions.key("direct-load.dup-action") + .enumType(ObLoadDupActionType.class) + .defaultValue(ObLoadDupActionType.REPLACE) + .withDescription("Action when there is duplicated record in direct load."); + + public static final ConfigOption DIRECT_LOAD_TIMEOUT = + ConfigOptions.key("direct-load.timeout") + .durationType() + .defaultValue(Duration.ofDays(7)) + .withDescription("Timeout for direct load task."); + + public static final ConfigOption DIRECT_LOAD_HEARTBEAT_TIMEOUT = + ConfigOptions.key("direct-load.heartbeat-timeout") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription("Client heartbeat timeout in direct load task."); + + public static final ConfigOption SYNC_WRITE = + ConfigOptions.key("sync-write") + .booleanType() + .defaultValue(false) + .withDescription("Whether to write synchronously."); + + public static final ConfigOption BUFFER_FLUSH_INTERVAL = + ConfigOptions.key("buffer-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription( + "The flush interval, over this time, asynchronous threads will flush data. Default value is '1s'. " + + "If it's set to zero value like '0', scheduled flushing will be disabled."); + + public static final ConfigOption BUFFER_SIZE = + ConfigOptions.key("buffer-flush.buffer-size") + .intType() + .defaultValue(1000) + .withDescription("Buffer size. Default value is '1000'."); + + public static final ConfigOption MAX_RETRIES = + ConfigOptions.key("max-retries") + .intType() + .defaultValue(3) + .withDescription( + "The max retry times if writing records to database failed. Default value is '3'."); +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java new file mode 100644 index 00000000000..f05b63e390e --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java @@ -0,0 +1,120 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.OperationType; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.utils.Preconditions; +import org.apache.flink.cdc.common.utils.SchemaUtils; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import com.oceanbase.connector.flink.table.Record; +import com.oceanbase.connector.flink.table.RecordSerializationSchema; + +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** A serializer for Event to Record. */ +public class OceanBaseEventSerializer implements RecordSerializationSchema { + + private ObjectMapper objectMapper = new ObjectMapper(); + private Map schemaMaps = new HashMap<>(); + + /** Format DATE type data. */ + public static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + /** Format timestamp-related type data. */ + public static final DateTimeFormatter DATE_TIME_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + + /** ZoneId from pipeline config to support timestamp with local time zone. */ + public final ZoneId pipelineZoneId; + + public OceanBaseEventSerializer(ZoneId zoneId) { + pipelineZoneId = zoneId; + } + + @Override + public Record serialize(Event event) { + if (event instanceof DataChangeEvent) { + return applyDataChangeEvent((DataChangeEvent) event); + } else if (event instanceof SchemaChangeEvent) { + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + TableId tableId = schemaChangeEvent.tableId(); + if (event instanceof CreateTableEvent) { + schemaMaps.put(tableId, ((CreateTableEvent) event).getSchema()); + } else { + if (!schemaMaps.containsKey(tableId)) { + throw new RuntimeException("schema of " + tableId + " is not existed."); + } + schemaMaps.put( + tableId, + SchemaUtils.applySchemaChangeEvent( + schemaMaps.get(tableId), schemaChangeEvent)); + } + } + return null; + } + + private Record applyDataChangeEvent(DataChangeEvent event) { + TableId tableId = event.tableId(); + Schema schema = schemaMaps.get(tableId); + Preconditions.checkNotNull(schema, event.tableId() + " is not existed"); + Map valueMap; + OperationType op = event.op(); + switch (op) { + case INSERT: + case UPDATE: + case REPLACE: + valueMap = serializerRecord(event.after(), schema); + // addDeleteSign(valueMap, false); + break; + case DELETE: + valueMap = serializerRecord(event.before(), schema); + // addDeleteSign(valueMap, true); + break; + default: + throw new UnsupportedOperationException("Unsupport Operation " + op); + } + + return null; + } + + /** serializer RecordData to Doris Value. */ + public Map serializerRecord(RecordData recordData, Schema schema) { + List columns = schema.getColumns(); + Map record = new HashMap<>(); + Preconditions.checkState( + columns.size() == recordData.getArity(), + "Column size does not match the data size"); + + return record; + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java new file mode 100644 index 00000000000..c14f8beafbb --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -0,0 +1,144 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseMySQLCatalog; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** Supports {@link OceanBaseDataSink} to schema evolution. */ +public class OceanBaseMetadataApplier implements MetadataApplier { + + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMetadataApplier.class); + + private final OceanBaseConnectorOptions connectorOptions; + private final Configuration config; + private final OceanBaseMySQLCatalog catalog; + + public OceanBaseMetadataApplier( + OceanBaseConnectorOptions connectorOptions, Configuration config) throws Exception { + this.connectorOptions = connectorOptions; + this.config = config; + this.catalog = (OceanBaseMySQLCatalog)OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); + } + + @Override + public void applySchemaChange(SchemaChangeEvent event) { + try { + // send schema change op to doris + if (event instanceof CreateTableEvent) { + applyCreateTableEvent((CreateTableEvent) event); + } else if (event instanceof AddColumnEvent) { + applyAddColumnEvent((AddColumnEvent) event); + } else if (event instanceof DropColumnEvent) { + applyDropColumnEvent((DropColumnEvent) event); + } else if (event instanceof RenameColumnEvent) { + applyRenameColumnEvent((RenameColumnEvent) event); + } else if (event instanceof AlterColumnTypeEvent) { + throw new RuntimeException("Unsupported schema change event, " + event); + } + } catch (Exception ex) { + throw new RuntimeException( + "Failed to schema change, " + event + ", reason: " + ex.getMessage()); + } + } + + private void applyCreateTableEvent(CreateTableEvent createTableEvent) + throws IllegalArgumentException { + Schema schema = createTableEvent.getSchema(); + TableId tableId = createTableEvent.tableId(); + OceanBaseTable oceanBaseTable = OceanBaseUtils.toOceanBaseTable(tableId, schema); + + if (!catalog.databaseExists(oceanBaseTable.getDatabaseName())) { + catalog.createDatabase(oceanBaseTable.getDatabaseName(), true); + } + + try { + catalog.createTable(oceanBaseTable, true); + LOG.info("Successful to create table, event: {}", createTableEvent); + } catch (OceanBaseCatalogException e) { + LOG.error("Failed to create table, event: {}", createTableEvent.tableId(), e); + throw new RuntimeException("Failed to create table, event: " + createTableEvent, e); + } + } + + + private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { + List addColumns = new ArrayList<>(); + for (AddColumnEvent.ColumnWithPosition columnWithPosition : + addColumnEvent.getAddedColumns()) { + // we will ignore position information, and always add the column to the last. + // The reason is that the order of columns between source table and OceanBase + // table may be not consistent because of limitations of OceanBase table, so the + // position may be meaningless. For example, primary keys of OceanBase table + // must be at the front, but mysql doest not have this limitation, so the order + // may be different, and also FIRST position is not allowed for OceanBase primary + // key table. + Column column = columnWithPosition.getAddColumn(); + OceanBaseColumn.Builder builder = + new OceanBaseColumn.Builder() + .setColumnName(column.getName()) + .setOrdinalPosition(-1) + .setColumnComment(column.getComment()); + OceanBaseUtils.toOceanBaseDataType(column, false, builder); + addColumns.add(builder.build()); + } + + TableId tableId = addColumnEvent.tableId(); + OceanBaseCatalogException alterException = null; + try { + catalog.alterAddColumns( + tableId.getSchemaName(), + tableId.getTableName(), + addColumns); + } catch (OceanBaseCatalogException e) { + alterException = e; + } + } + + private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { + // TODO OceanBase plans to support column drop since 3.3 which has not been released. + // Support it later. + throw new UnsupportedOperationException("Rename column is not supported currently"); + } + + private void applyRenameColumnEvent(RenameColumnEvent renameColumnEvent) { + // TODO OceanBase plans to support column rename since 3.3 which has not been released. + // Support it later. + throw new UnsupportedOperationException("Rename column is not supported currently"); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java new file mode 100644 index 00000000000..58255d5b28a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -0,0 +1,382 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.BigIntType; +import org.apache.flink.cdc.common.types.BooleanType; +import org.apache.flink.cdc.common.types.CharType; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypeDefaultVisitor; +import org.apache.flink.cdc.common.types.DateType; +import org.apache.flink.cdc.common.types.DecimalType; +import org.apache.flink.cdc.common.types.DoubleType; +import org.apache.flink.cdc.common.types.FloatType; +import org.apache.flink.cdc.common.types.IntType; +import org.apache.flink.cdc.common.types.LocalZonedTimestampType; +import org.apache.flink.cdc.common.types.SmallIntType; +import org.apache.flink.cdc.common.types.TimestampType; +import org.apache.flink.cdc.common.types.TinyIntType; +import org.apache.flink.cdc.common.types.VarCharType; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; + +import java.time.LocalDate; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision; +import static org.apache.flink.cdc.common.types.DataTypeChecks.getScale; + +/** Utilities for conversion from source table to OceanBase table. */ +public class OceanBaseUtils { + + /** Convert a source table to {@link OceanBaseTable}. */ + public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { + if (schema.primaryKeys().isEmpty()) { + throw new RuntimeException( + String.format( + "Only support OceanBase primary key table, but the source table %s has no primary keys", + tableId)); + } + + // For OceanBase primary key table DDL, primary key columns must be defined before other + // columns, + // so reorder the columns in the source schema to make primary key columns at the front + List orderedColumns = new ArrayList<>(); + for (String primaryKey : schema.primaryKeys()) { + orderedColumns.add(schema.getColumn(primaryKey).get()); + } + for (Column column : schema.getColumns()) { + if (!schema.primaryKeys().contains(column.getName())) { + orderedColumns.add(column); + } + } + + int primaryKeyCount = schema.primaryKeys().size(); + List oceanBaseColumns = new ArrayList<>(); + for (int i = 0; i < orderedColumns.size(); i++) { + Column column = orderedColumns.get(i); + OceanBaseColumn.Builder builder = + new OceanBaseColumn.Builder() + .setColumnName(column.getName()) + .setOrdinalPosition(i) + .setColumnComment(column.getComment()); + toOceanBaseDataType(column, i < primaryKeyCount, builder); + oceanBaseColumns.add(builder.build()); + } + + OceanBaseTable.Builder tableBuilder = + new OceanBaseTable.Builder() + .setDatabaseName(tableId.getSchemaName()) + .setTableName(tableId.getTableName()) + .setTableType(OceanBaseTable.TableType.PRIMARY_KEY) + .setColumns(oceanBaseColumns) + .setTableKeys(schema.primaryKeys()) + // use primary keys as distribution keys by default + .setDistributionKeys(schema.primaryKeys()) + .setComment(schema.comment()); + return tableBuilder.build(); + } + + /** Convert CDC data type to OceanBase data type. */ + public static void toOceanBaseDataType( + Column cdcColumn, boolean isPrimaryKeys, OceanBaseColumn.Builder builder) { + CdcDataTypeTransformer dataTypeTransformer = + new CdcDataTypeTransformer(isPrimaryKeys, builder); + cdcColumn.getType().accept(dataTypeTransformer); + } + + /** Format DATE type data. */ + private static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + /** Format timestamp-related type data. */ + private static final DateTimeFormatter DATETIME_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + + /** + * Creates an accessor for getting elements in an internal RecordData structure at the given + * position. + * + * @param fieldType the element type of the RecordData + * @param fieldPos the element position of the RecordData + * @param zoneId the time zone used when converting from TIMESTAMP WITH LOCAL TIME ZONE + * + */ + public static RecordData.FieldGetter createFieldGetter( + DataType fieldType, int fieldPos, ZoneId zoneId) { + final RecordData.FieldGetter fieldGetter; + // ordered by type root definition + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + fieldGetter = record -> record.getBoolean(fieldPos); + break; + case TINYINT: + fieldGetter = record -> record.getByte(fieldPos); + break; + case SMALLINT: + fieldGetter = record -> record.getShort(fieldPos); + break; + case INTEGER: + fieldGetter = record -> record.getInt(fieldPos); + break; + case BIGINT: + fieldGetter = record -> record.getLong(fieldPos); + break; + case FLOAT: + fieldGetter = record -> record.getFloat(fieldPos); + break; + case DOUBLE: + fieldGetter = record -> record.getDouble(fieldPos); + break; + case DECIMAL: + final int decimalPrecision = getPrecision(fieldType); + final int decimalScale = getScale(fieldType); + fieldGetter = + record -> + record.getDecimal(fieldPos, decimalPrecision, decimalScale) + .toBigDecimal(); + break; + case CHAR: + case VARCHAR: + fieldGetter = record -> record.getString(fieldPos).toString(); + break; + case DATE: + fieldGetter = + record -> + LocalDate.ofEpochDay(record.getInt(fieldPos)) + .format(DATE_FORMATTER); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + fieldGetter = + record -> + record.getTimestamp(fieldPos, getPrecision(fieldType)) + .toLocalDateTime() + .format(DATETIME_FORMATTER); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + fieldGetter = + record -> + ZonedDateTime.ofInstant( + record.getLocalZonedTimestampData( + fieldPos, getPrecision(fieldType)) + .toInstant(), + zoneId) + .toLocalDateTime() + .format(DATETIME_FORMATTER); + break; + default: + throw new UnsupportedOperationException( + "Don't support data type " + fieldType.getTypeRoot()); + } + if (!fieldType.isNullable()) { + return fieldGetter; + } + return row -> { + if (row.isNullAt(fieldPos)) { + return null; + } + return fieldGetter.getFieldOrNull(row); + }; + } + + // ------------------------------------------------------------------------------------------ + // OceanBase data types + // ------------------------------------------------------------------------------------------ + + 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 = "BIGINT UNSIGNED"; + public static final String FLOAT = "FLOAT"; + public static final String DOUBLE = "DOUBLE"; + public static final String DECIMAL = "DECIMAL"; + public static final String CHAR = "CHAR"; + public static final String VARCHAR = "VARCHAR"; + public static final String STRING = "STRING"; + public static final String DATE = "DATE"; + public static final String DATETIME = "DATETIME"; + public static final String JSON = "JSON"; + + /** Max size of char type of OceanBase. */ + public static final int MAX_CHAR_SIZE = 255; + + /** Max size of varchar type of OceanBase. */ + public static final int MAX_VARCHAR_SIZE = 1048576; + + /** Transforms CDC {@link DataType} to OceanBase data type. */ + public static class CdcDataTypeTransformer + extends DataTypeDefaultVisitor { + + private final OceanBaseColumn.Builder builder; + private final boolean isPrimaryKeys; + + public CdcDataTypeTransformer(boolean isPrimaryKeys, OceanBaseColumn.Builder builder) { + this.isPrimaryKeys = isPrimaryKeys; + this.builder = builder; + } + + @Override + public OceanBaseColumn.Builder visit(BooleanType booleanType) { + builder.setDataType(BOOLEAN); + builder.setNullable(booleanType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(TinyIntType tinyIntType) { + builder.setDataType(TINYINT); + builder.setNullable(tinyIntType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(SmallIntType smallIntType) { + builder.setDataType(SMALLINT); + builder.setNullable(smallIntType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(IntType intType) { + builder.setDataType(INT); + builder.setNullable(intType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(BigIntType bigIntType) { + builder.setDataType(BIGINT); + builder.setNullable(bigIntType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(FloatType floatType) { + builder.setDataType(FLOAT); + builder.setNullable(floatType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(DoubleType doubleType) { + builder.setDataType(DOUBLE); + builder.setNullable(doubleType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(DecimalType decimalType) { + // OceanBase does not support Decimal as primary key, so decimal should be cast to + // VARCHAR. + if (!isPrimaryKeys) { + builder.setDataType(DECIMAL); + builder.setColumnSize(decimalType.getPrecision()); + builder.setDecimalDigits(decimalType.getScale()); + } else { + builder.setDataType(VARCHAR); + // For a DecimalType with precision N, we may need N + 1 or N + 2 characters to + // store it as a + // string (one for negative sign, and one for decimal point) + builder.setColumnSize( + Math.min( + decimalType.getScale() != 0 + ? decimalType.getPrecision() + 2 + : decimalType.getPrecision() + 1, + MAX_VARCHAR_SIZE)); + } + builder.setNullable(decimalType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(CharType charType) { + // CDC and OceanBase use different units for the length. It's the number + // of characters in CDC, and the number of bytes in OceanBase. One chinese + // character will use 3 bytes because it uses UTF-8, so the length of OceanBase + // char type should be three times as that of CDC char type. Specifically, if + // the length of OceanBase exceeds the MAX_CHAR_SIZE, map CDC char type to OceanBase + // varchar type + int length = charType.getLength(); + long oceanBaseLength = length * 3L; + // In the OceanBase, The primary key columns can be any of the following data types: + // BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, STRING, VARCHAR, DATE, and + // DATETIME, But it doesn't include CHAR. When a char type appears in the primary key of + // MySQL, creating a table in OceanBase requires conversion to varchar type. + if (oceanBaseLength <= MAX_CHAR_SIZE && !isPrimaryKeys) { + builder.setDataType(CHAR); + builder.setNullable(charType.isNullable()); + builder.setColumnSize((int) oceanBaseLength); + } else { + builder.setDataType(VARCHAR); + builder.setNullable(charType.isNullable()); + builder.setColumnSize((int) Math.min(oceanBaseLength, MAX_VARCHAR_SIZE)); + } + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(VarCharType varCharType) { + // CDC and OceanBase use different units for the length. It's the number + // of characters in CDC, and the number of bytes in OceanBase. One chinese + // character will use 3 bytes because it uses UTF-8, so the length of OceanBase + // varchar type should be three times as that of CDC varchar type. + int length = varCharType.getLength(); + long oceanBaseLength = length * 3L; + builder.setDataType(VARCHAR); + builder.setNullable(varCharType.isNullable()); + builder.setColumnSize((int) Math.min(oceanBaseLength, MAX_VARCHAR_SIZE)); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(DateType dateType) { + builder.setDataType(DATE); + builder.setNullable(dateType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(TimestampType timestampType) { + builder.setDataType(DATETIME); + builder.setNullable(timestampType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(LocalZonedTimestampType localZonedTimestampType) { + builder.setDataType(DATETIME); + builder.setNullable(localZonedTimestampType.isNullable()); + return builder; + } + + @Override + protected OceanBaseColumn.Builder defaultMethod(DataType dataType) { + throw new UnsupportedOperationException("Unsupported CDC data type " + dataType); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory new file mode 100644 index 00000000000..42dec8b24ab --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/resources/META-INF/services/org.apache.flink.cdc.common.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.cdc.connectors.oceanbase.factory.OceanBaseDataSinkFactory diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java new file mode 100644 index 00000000000..9ceb6f08df7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java @@ -0,0 +1,130 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase; + +import org.testcontainers.containers.JdbcDatabaseContainer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; + +public class OceanBaseContainer extends JdbcDatabaseContainer { + + public static final String DOCKER_IMAGE_NAME = "oceanbase/oceanbase-ce"; + + private static final DockerImageName DEFAULT_IMAGE_NAME = + DockerImageName.parse(DOCKER_IMAGE_NAME); + + private static final Integer SQL_PORT = 2881; + private static final Integer RPC_PORT = 2882; + + private static final String DEFAULT_USERNAME = "root"; + private static final String DEFAULT_PASSWORD = ""; + private static final String DEFAULT_TENANT_NAME = "test"; + private static final String DEFAULT_DATABASE_NAME = "test"; + + private String sysPassword = DEFAULT_PASSWORD; + + public OceanBaseContainer(String dockerImageName) { + this(DockerImageName.parse(dockerImageName)); + } + + public OceanBaseContainer(DockerImageName dockerImageName) { + super(dockerImageName); + dockerImageName.assertCompatibleWith(DEFAULT_IMAGE_NAME); + + this.waitStrategy = + Wait.forLogMessage(".*boot success!.*", 1) + .withStartupTimeout(Duration.ofMinutes(5)); + + addExposedPorts(SQL_PORT, RPC_PORT); + } + + @Override + public String getDriverClassName() { + return "com.oceanbase.jdbc.Driver"; + } + + public Integer getSqlPort() { + return getActualPort(SQL_PORT); + } + + public Integer getActualPort(int port) { + return "host".equals(getNetworkMode()) ? port : getMappedPort(port); + } + + @Override + public String getJdbcUrl() { + return getJdbcUrl(DEFAULT_DATABASE_NAME); + } + + public String getJdbcUrl(String databaseName) { + String additionalUrlParams = constructUrlParameters("?", "&"); + return "jdbc:mysql://" + + getHost() + + ":" + + getSqlPort() + + "/" + + databaseName + + additionalUrlParams; + } + + public OceanBaseContainer withSysPassword(String sysPassword) { + this.sysPassword = sysPassword; + return this; + } + + public String getSysUsername() { + return DEFAULT_USERNAME; + } + + public String getSysPassword() { + return sysPassword; + } + + @Override + public String getDatabaseName() { + return DEFAULT_DATABASE_NAME; + } + + @Override + public String getUsername() { + return DEFAULT_USERNAME + "@" + DEFAULT_TENANT_NAME; + } + + @Override + public String getPassword() { + return DEFAULT_PASSWORD; + } + + @Override + protected String getTestQueryString() { + return "SELECT 1"; + } + + @Override + protected void waitUntilContainerStarted() { + getWaitStrategy().waitUntilReady(this); + } + + @Override + protected void configure() { + withEnv("MODE", "slim"); + withEnv("OB_ROOT_PASSWORD", sysPassword); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java new file mode 100644 index 00000000000..2fbabfcc6d2 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java @@ -0,0 +1,50 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase; + +import org.junit.ClassRule; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.MountableFile; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +public class OceanBaseTestBase { + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestBase.class); + + public static final String IMAGE_TAG = "4.2.1_bp2"; + + @ClassRule + public static final OceanBaseContainer OB_SERVER = + new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) + .withNetworkMode("host") + .withSysPassword("123456") + .withCopyFileToContainer( + MountableFile.forClasspathResource("sql/init.sql"), + "/root/boot/init.d/init.sql") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + protected Connection getConnection() throws SQLException { + return DriverManager.getConnection(OB_SERVER.getJdbcUrl(), OB_SERVER.getUsername(), OB_SERVER.getPassword()); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java new file mode 100644 index 00000000000..e08947a1b16 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -0,0 +1,36 @@ +package org.apache.flink.cdc.connectors.oceanbase.catalog; + +import com.google.common.collect.Lists; +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; +import org.junit.Ignore; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +/** Tests for {@link OceanBaseMySQLCatalogTest}. */ +@Ignore +public class OceanBaseMySQLCatalogTest { + + public static final ImmutableMap configMap = ImmutableMap.builder() + .put("url", System.getenv("url")) + .put("username", System.getenv("username")) + .put("password", System.getenv("password")) + .build(); + + @Test + public void testBuildAlterAddColumnsSql() { + OceanBaseMySQLCatalog oceanBaseCatalog = new OceanBaseMySQLCatalog(new OceanBaseConnectorOptions(configMap)); + + List addColumns = Lists.newArrayList(); + addColumns.add(new OceanBaseColumn.Builder() + .setColumnName("age") + .setOrdinalPosition(-1) + .setColumnComment("age") + .setDataType("varchar(10)") + .build()); + String columnsSql = oceanBaseCatalog.buildAlterAddColumnsSql("test", "test", addColumns); + Assertions.assertEquals("ALTER TABLE `test`.`test` ADD COLUMN `age` VARCHAR(10) NULL COMMENT \"age\";", columnsSql); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/resources/sql/init.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/resources/sql/init.sql new file mode 100644 index 00000000000..b488deb4b15 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/resources/sql/init.sql @@ -0,0 +1,39 @@ +-- 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. + +CREATE DATABASE IF NOT EXISTS test; +USE test; + +CREATE TABLE products +( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + description VARCHAR(512), + weight DECIMAL(20, 10) +); + + +CREATE TABLE gis_types +( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + point_c POINT, + geometry_c GEOMETRY, + linestring_c LINESTRING, + polygon_c POLYGON, + multipoint_c MULTIPOINT, + multiline_c MULTILINESTRING, + multipolygon_c MULTIPOLYGON, + geometrycollection_c GEOMETRYCOLLECTION +) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml index 424372f4c1c..77a6e8b3fa6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/pom.xml @@ -34,6 +34,7 @@ limitations under the License. flink-cdc-pipeline-connector-starrocks flink-cdc-pipeline-connector-kafka flink-cdc-pipeline-connector-paimon + flink-cdc-pipeline-connector-oceanbase From 0647ecb1f28f236f744a0790ea4a4be0af4fcd5c Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 17 May 2024 16:42:06 +0800 Subject: [PATCH 02/23] Enhancement: Refactor and improve each module --- .../pom.xml | 7 - .../catalog/OceanBaseCatalogException.java | 1 + .../catalog/OceanBaseCatalogFactory.java | 26 ++- .../catalog/OceanBaseMySQLCatalog.java | 30 ++- .../catalog/OceanBaseOracleCatalog.java | 8 +- .../factory/OceanBaseDataSinkFactory.java | 3 + .../oceanbase/sink/MySQLDialect.java | 21 --- .../oceanbase/sink/OceanBaseDataSink.java | 3 +- .../sink/OceanBaseDataSinkOptions.java | 1 + ...=> OceanBaseEventSerializationSchema.java} | 64 ++++--- .../sink/OceanBaseMetadataApplier.java | 18 +- .../oceanbase/sink/OceanBaseRowConvert.java | 176 ++++++++++++++++++ .../oceanbase/OceanBaseContainer.java | 1 + .../oceanbase/OceanBaseTestBase.java | 6 +- .../catalog/OceanBaseMySQLCatalogTest.java | 53 ++++-- 15 files changed, 320 insertions(+), 98 deletions(-) delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java rename flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/{OceanBaseEventSerializer.java => OceanBaseEventSerializationSchema.java} (65%) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index 46969495313..7a60c107355 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -48,13 +48,6 @@ limitations under the License. 2.4.8 test - - - - - - - org.testcontainers jdbc diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java index ed12088f465..6d87a59f6f2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogException.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; +/** A catalog-related, runtime exception. */ public class OceanBaseCatalogException extends RuntimeException { /** @param message the detail message. */ public OceanBaseCatalogException(String message) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java index 45ff4f52afa..81f49c093b6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java @@ -1,3 +1,20 @@ +/* + * 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. + */ + package org.apache.flink.cdc.connectors.oceanbase.catalog; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; @@ -5,19 +22,20 @@ import com.oceanbase.connector.flink.dialect.OceanBaseDialect; import com.oceanbase.connector.flink.dialect.OceanBaseMySQLDialect; import com.oceanbase.connector.flink.dialect.OceanBaseOracleDialect; -import org.apache.flink.cdc.common.factories.DataSinkFactory; /** A {@link OceanBaseCatalogFactory} to create {@link OceanBaseCatalog}. */ public class OceanBaseCatalogFactory { - public static OceanBaseCatalog createOceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) throws Exception { - try (OceanBaseConnectionProvider connectionProvider = new OceanBaseConnectionProvider(connectorOptions)) { + public static OceanBaseCatalog createOceanBaseCatalog( + OceanBaseConnectorOptions connectorOptions) throws Exception { + try (OceanBaseConnectionProvider connectionProvider = + new OceanBaseConnectionProvider(connectorOptions)) { OceanBaseDialect dialect = connectionProvider.getDialect(); if (dialect instanceof OceanBaseMySQLDialect) { return new OceanBaseMySQLCatalog(connectorOptions); } else if (dialect instanceof OceanBaseOracleDialect) { return new OceanBaseOracleCatalog(connectorOptions); - }else { + } else { throw new OceanBaseCatalogException("This tenant is not supported currently"); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index 53b56ec45df..566b623c16c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -28,7 +28,10 @@ import java.util.Optional; import java.util.stream.Collectors; -/** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under MySQL mode. */ +/** + * A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under MySQL + * mode. + */ public class OceanBaseMySQLCatalog extends OceanBaseCatalog { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLCatalog.class); @@ -126,7 +129,8 @@ public void createTable(OceanBaseTable table, boolean ignoreIfExists) } } - public void alterAddColumns(String databaseName, String tableName, List addColumns) { + public void alterAddColumns( + String databaseName, String tableName, List addColumns) { Preconditions.checkArgument( !StringUtils.isNullOrWhitespaceOnly(databaseName), "database name cannot be null or empty."); @@ -135,15 +139,23 @@ public void alterAddColumns(String databaseName, String tableName, List addColumns) { + String databaseName, String tableName, List addColumns) { StringBuilder builder = new StringBuilder(); builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); String columnsStmt = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java index 76f3cf73b34..1fb1bfd0f8e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java @@ -17,9 +17,10 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; -import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,7 +28,10 @@ import java.util.Optional; import java.util.stream.Collectors; -/** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under Oracle mode. */ +/** + * A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under Oracle + * mode. + */ public class OceanBaseOracleCatalog extends OceanBaseCatalog { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseOracleCatalog.class); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index 3ab247dfa49..5884ec6c290 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.oceanbase.factory; +import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.factories.DataSinkFactory; @@ -31,6 +32,8 @@ import java.util.HashSet; import java.util.Set; +/** A {@link DataSinkFactory} to create {@link OceanBaseDataSink}. */ +@Internal public class OceanBaseDataSinkFactory implements DataSinkFactory { @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java deleted file mode 100644 index 834aba557b7..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/MySQLDialect.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -package org.apache.flink.cdc.connectors.oceanbase.sink; - -/*MySQL Dialect*/ -public class MySQLDialect {} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java index 890f952c43a..bf912395887 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -33,6 +33,7 @@ import java.io.Serializable; import java.time.ZoneId; +/** A {@link DataSink} for "OceanBase" pipeline connector. */ public class OceanBaseDataSink implements DataSink, Serializable { private final OceanBaseConnectorOptions connectorOptions; @@ -63,7 +64,7 @@ public EventSinkProvider getEventSinkProvider() { new OceanBaseSink<>( connectorOptions, null, - new OceanBaseEventSerializer(zoneId), + new OceanBaseEventSerializationSchema(zoneId), DataChangeRecord.KeyExtractor.simple(), recordFlusher)); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java index 96dba6a53d5..c6183ed59a0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java @@ -24,6 +24,7 @@ import java.time.Duration; +/** Options for {@link OceanBaseDataSink}. */ public class OceanBaseDataSinkOptions { // ------------------------------------------------------------------------------------------ // Options for sink connector diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java similarity index 65% rename from flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java rename to flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index f05b63e390e..969ac4cea4b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -29,35 +29,26 @@ import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.common.utils.SchemaUtils; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; +import com.oceanbase.connector.flink.table.DataChangeRecord; import com.oceanbase.connector.flink.table.Record; import com.oceanbase.connector.flink.table.RecordSerializationSchema; +import com.oceanbase.connector.flink.table.TableInfo; import java.time.ZoneId; -import java.time.format.DateTimeFormatter; import java.util.HashMap; import java.util.List; import java.util.Map; /** A serializer for Event to Record. */ -public class OceanBaseEventSerializer implements RecordSerializationSchema { - - private ObjectMapper objectMapper = new ObjectMapper(); +public class OceanBaseEventSerializationSchema implements RecordSerializationSchema { private Map schemaMaps = new HashMap<>(); - /** Format DATE type data. */ - public static final DateTimeFormatter DATE_FORMATTER = - DateTimeFormatter.ofPattern("yyyy-MM-dd"); - - /** Format timestamp-related type data. */ - public static final DateTimeFormatter DATE_TIME_FORMATTER = - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - /** ZoneId from pipeline config to support timestamp with local time zone. */ public final ZoneId pipelineZoneId; - public OceanBaseEventSerializer(ZoneId zoneId) { + public OceanBaseEventSerializationSchema(ZoneId zoneId) { pipelineZoneId = zoneId; } @@ -87,34 +78,61 @@ private Record applyDataChangeEvent(DataChangeEvent event) { TableId tableId = event.tableId(); Schema schema = schemaMaps.get(tableId); Preconditions.checkNotNull(schema, event.tableId() + " is not existed"); - Map valueMap; + Object[] values; OperationType op = event.op(); + boolean isDelete = false; switch (op) { case INSERT: case UPDATE: case REPLACE: - valueMap = serializerRecord(event.after(), schema); - // addDeleteSign(valueMap, false); + values = serializerRecord(event.after(), schema); break; case DELETE: - valueMap = serializerRecord(event.before(), schema); - // addDeleteSign(valueMap, true); + values = serializerRecord(event.before(), schema); + isDelete = true; break; default: throw new UnsupportedOperationException("Unsupport Operation " + op); } - return null; + return buildDataChangeRecord(tableId, schema, values, isDelete); + } + + private DataChangeRecord buildDataChangeRecord( + TableId tableId, Schema schema, Object[] values, boolean isDelete) { + com.oceanbase.connector.flink.table.TableId oceanBaseTableId = + new com.oceanbase.connector.flink.table.TableId( + tableId.getSchemaName(), tableId.getTableName()); + TableInfo tableInfo = + new TableInfo( + oceanBaseTableId, + schema.partitionKeys(), + schema.getColumnNames(), + Lists.newArrayList(), + null); + + return new DataChangeRecord( + tableInfo, + isDelete ? DataChangeRecord.Type.DELETE : DataChangeRecord.Type.UPSERT, + values); } /** serializer RecordData to Doris Value. */ - public Map serializerRecord(RecordData recordData, Schema schema) { + public Object[] serializerRecord(RecordData recordData, Schema schema) { List columns = schema.getColumns(); - Map record = new HashMap<>(); Preconditions.checkState( columns.size() == recordData.getArity(), "Column size does not match the data size"); + Object[] values = new Object[columns.size()]; + + for (int i = 0; i < recordData.getArity(); i++) { + OceanBaseRowConvert.SerializationConverter converter = + OceanBaseRowConvert.createNullableExternalConverter( + columns.get(i).getType(), pipelineZoneId); + Object field = converter.serialize(i, recordData); + values[i] = field; + } - return record; + return values; } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index c14f8beafbb..fd72c8997db 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; -import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.AddColumnEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; @@ -34,6 +33,8 @@ import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseMySQLCatalog; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +54,9 @@ public OceanBaseMetadataApplier( OceanBaseConnectorOptions connectorOptions, Configuration config) throws Exception { this.connectorOptions = connectorOptions; this.config = config; - this.catalog = (OceanBaseMySQLCatalog)OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); + this.catalog = + (OceanBaseMySQLCatalog) + OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); } @Override @@ -96,7 +99,6 @@ private void applyCreateTableEvent(CreateTableEvent createTableEvent) } } - private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { List addColumns = new ArrayList<>(); for (AddColumnEvent.ColumnWithPosition columnWithPosition : @@ -119,15 +121,7 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { } TableId tableId = addColumnEvent.tableId(); - OceanBaseCatalogException alterException = null; - try { - catalog.alterAddColumns( - tableId.getSchemaName(), - tableId.getTableName(), - addColumns); - } catch (OceanBaseCatalogException e) { - alterException = e; - } + catalog.alterAddColumns(tableId.getSchemaName(), tableId.getTableName(), addColumns); } private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java new file mode 100644 index 00000000000..b42d2fc1ed9 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java @@ -0,0 +1,176 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.GenericArrayData; +import org.apache.flink.cdc.common.data.GenericMapData; +import org.apache.flink.cdc.common.data.MapData; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.types.DataField; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypeChecks; +import org.apache.flink.cdc.common.types.DecimalType; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.common.types.ZonedTimestampType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.io.Serializable; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** converter {@link RecordData} type object to oceanbase field. */ +public class OceanBaseRowConvert implements Serializable { + private static final long serialVersionUID = 1L; + private static final ObjectMapper objectMapper = new ObjectMapper(); + + /** Runtime converter to convert {@link RecordData} type object to oceanbase field. */ + @FunctionalInterface + interface SerializationConverter extends Serializable { + Object serialize(int index, RecordData field); + } + + static SerializationConverter createNullableExternalConverter( + DataType type, ZoneId pipelineZoneId) { + return wrapIntoNullableExternalConverter(createExternalConverter(type, pipelineZoneId)); + } + + static SerializationConverter wrapIntoNullableExternalConverter( + SerializationConverter serializationConverter) { + return (index, val) -> { + if (val == null || val.isNullAt(index)) { + return null; + } else { + return serializationConverter.serialize(index, val); + } + }; + } + + static SerializationConverter createExternalConverter(DataType type, ZoneId pipelineZoneId) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + return (index, val) -> val.getString(index).toString(); + case BOOLEAN: + return (index, val) -> val.getBoolean(index); + case BINARY: + case VARBINARY: + return (index, val) -> val.getBinary(index); + case DECIMAL: + final int decimalPrecision = ((DecimalType) type).getPrecision(); + final int decimalScale = ((DecimalType) type).getScale(); + return (index, val) -> + val.getDecimal(index, decimalPrecision, decimalScale).toBigDecimal(); + case TINYINT: + return (index, val) -> val.getByte(index); + case SMALLINT: + return (index, val) -> val.getShort(index); + case INTEGER: + return (index, val) -> val.getInt(index); + case BIGINT: + return (index, val) -> val.getLong(index); + case FLOAT: + return (index, val) -> val.getFloat(index); + case DOUBLE: + return (index, val) -> val.getDouble(index); + case DATE: + return (index, val) -> Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (index, val) -> + val.getTimestamp(index, DataTypeChecks.getPrecision(type)).toTimestamp(); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (index, val) -> + Timestamp.valueOf( + ZonedDateTime.ofInstant( + val.getLocalZonedTimestampData( + index, + DataTypeChecks.getPrecision(type)) + .toInstant(), + pipelineZoneId) + .toLocalDateTime()); + case TIMESTAMP_WITH_TIME_ZONE: + final int zonedP = ((ZonedTimestampType) type).getPrecision(); + return (index, val) -> val.getTimestamp(index, zonedP).toTimestamp(); + case ARRAY: + return (index, val) -> convertArrayData(val.getArray(index), type); + case MAP: + return (index, val) -> writeValueAsString(convertMapData(val.getMap(index), type)); + case ROW: + return (index, val) -> + writeValueAsString(convertRowData(val, index, type, pipelineZoneId)); + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } + + private static String convertArrayData(ArrayData array, DataType type) { + if (array instanceof GenericArrayData) { + return IntStream.range(0, array.size()) + .mapToObj(i -> array.getString(i).toString()) + .collect(Collectors.joining(",")); + } + throw new UnsupportedOperationException("Unsupported array data: " + array.getClass()); + } + + private static Object convertMapData(MapData map, DataType type) { + Map result = new HashMap<>(); + if (map instanceof GenericMapData) { + GenericMapData gMap = (GenericMapData) map; + for (Object key : ((GenericArrayData) gMap.keyArray()).toObjectArray()) { + result.put(key, gMap.get(key)); + } + return result; + } + throw new UnsupportedOperationException("Unsupported map data: " + map.getClass()); + } + + private static Object convertRowData( + RecordData val, int index, DataType type, ZoneId pipelineZoneId) { + RowType rowType = (RowType) type; + Map value = new HashMap<>(); + RecordData row = val.getRow(index, rowType.getFieldCount()); + + List fields = rowType.getFields(); + for (int i = 0; i < fields.size(); i++) { + DataField rowField = fields.get(i); + SerializationConverter converter = + createNullableExternalConverter(rowField.getType(), pipelineZoneId); + Object valTmp = converter.serialize(i, row); + value.put(rowField.getName(), valTmp.toString()); + } + return value; + } + + private static String writeValueAsString(Object object) { + try { + return objectMapper.writeValueAsString(object); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java index 9ceb6f08df7..a952853ac8a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java @@ -23,6 +23,7 @@ import java.time.Duration; +/** A {@link org.testcontainers.containers.Container} for "OceanBase". */ public class OceanBaseContainer extends JdbcDatabaseContainer { public static final String DOCKER_IMAGE_NAME = "oceanbase/oceanbase-ce"; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java index 2fbabfcc6d2..eb95c50a088 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java @@ -18,7 +18,6 @@ package org.apache.flink.cdc.connectors.oceanbase; import org.junit.ClassRule; -import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -27,8 +26,8 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -import java.sql.Statement; +/** Basic class for testing OceanBase. */ public class OceanBaseTestBase { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestBase.class); @@ -45,6 +44,7 @@ public class OceanBaseTestBase { .withLogConsumer(new Slf4jLogConsumer(LOG)); protected Connection getConnection() throws SQLException { - return DriverManager.getConnection(OB_SERVER.getJdbcUrl(), OB_SERVER.getUsername(), OB_SERVER.getPassword()); + return DriverManager.getConnection( + OB_SERVER.getJdbcUrl(), OB_SERVER.getUsername(), OB_SERVER.getPassword()); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java index e08947a1b16..e6eea196e96 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -1,8 +1,26 @@ +/* + * 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. + */ + package org.apache.flink.cdc.connectors.oceanbase.catalog; -import com.google.common.collect.Lists; -import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.junit.Ignore; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -13,24 +31,29 @@ @Ignore public class OceanBaseMySQLCatalogTest { - public static final ImmutableMap configMap = ImmutableMap.builder() - .put("url", System.getenv("url")) - .put("username", System.getenv("username")) - .put("password", System.getenv("password")) - .build(); + private static final ImmutableMap configMap = + ImmutableMap.builder() + .put("url", System.getenv("url")) + .put("username", System.getenv("username")) + .put("password", System.getenv("password")) + .build(); @Test public void testBuildAlterAddColumnsSql() { - OceanBaseMySQLCatalog oceanBaseCatalog = new OceanBaseMySQLCatalog(new OceanBaseConnectorOptions(configMap)); + OceanBaseMySQLCatalog oceanBaseCatalog = + new OceanBaseMySQLCatalog(new OceanBaseConnectorOptions(configMap)); List addColumns = Lists.newArrayList(); - addColumns.add(new OceanBaseColumn.Builder() - .setColumnName("age") - .setOrdinalPosition(-1) - .setColumnComment("age") - .setDataType("varchar(10)") - .build()); + addColumns.add( + new OceanBaseColumn.Builder() + .setColumnName("age") + .setOrdinalPosition(-1) + .setColumnComment("age") + .setDataType("varchar(10)") + .build()); String columnsSql = oceanBaseCatalog.buildAlterAddColumnsSql("test", "test", addColumns); - Assertions.assertEquals("ALTER TABLE `test`.`test` ADD COLUMN `age` VARCHAR(10) NULL COMMENT \"age\";", columnsSql); + Assertions.assertEquals( + "ALTER TABLE `test`.`test` ADD COLUMN `age` VARCHAR(10) NULL COMMENT \"age\";", + columnsSql); } } From 294a53735b0d8b8a2bf65bb8fcef0f4c1b5b0060 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 17 May 2024 17:55:43 +0800 Subject: [PATCH 03/23] Enhancement: add test for OceanBaseCatalogFactory --- .../pom.xml | 6 ++ .../factory/OceanBaseDataSinkFactory.java | 16 +++--- .../oceanbase/sink/OceanBaseDataSink.java | 6 -- .../catalog/OceanBaseMySQLCatalogTest.java | 13 +++-- .../sink/OceanBaseDataSinkFactoryTest.java | 55 +++++++++++++++++++ 5 files changed, 76 insertions(+), 20 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index 7a60c107355..db514dbba15 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -54,5 +54,11 @@ limitations under the License. ${testcontainers.version} test + + org.apache.flink + flink-cdc-composer + ${project.version} + test + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index 5884ec6c290..83339cae2d3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -21,7 +21,6 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.factories.DataSinkFactory; -import org.apache.flink.cdc.common.pipeline.PipelineOptions; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSink; import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSinkOptions; @@ -32,6 +31,8 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.cdc.common.pipeline.PipelineOptions.PIPELINE_LOCAL_TIME_ZONE; + /** A {@link DataSinkFactory} to create {@link OceanBaseDataSink}. */ @Internal public class OceanBaseDataSinkFactory implements DataSinkFactory { @@ -40,13 +41,12 @@ public class OceanBaseDataSinkFactory implements DataSinkFactory { public DataSink createDataSink(Context context) { Configuration config = context.getFactoryConfiguration(); OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(config.toMap()); - - return new OceanBaseDataSink( - connectorOptions, - config, - ZoneId.of( - context.getPipelineConfiguration() - .get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE))); + String zoneStr = context.getFactoryConfiguration().get(PIPELINE_LOCAL_TIME_ZONE); + ZoneId zoneId = + PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(zoneStr) + ? ZoneId.systemDefault() + : ZoneId.of(zoneStr); + return new OceanBaseDataSink(connectorOptions, config, zoneId); } @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java index bf912395887..49da92c6af7 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -28,7 +28,6 @@ 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.TableId; import java.io.Serializable; import java.time.ZoneId; @@ -53,11 +52,6 @@ public OceanBaseDataSink( public EventSinkProvider getEventSinkProvider() { OceanBaseConnectionProvider connectionProvider = new OceanBaseConnectionProvider(connectorOptions); - TableId tableId = - new TableId( - connectionProvider.getDialect()::getFullTableName, - connectorOptions.getSchemaName(), - connectorOptions.getTableName()); OceanBaseRecordFlusher recordFlusher = new OceanBaseRecordFlusher(connectorOptions, connectionProvider); return FlinkSinkProvider.of( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java index e6eea196e96..54967deee7c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -17,25 +17,26 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; +import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; + import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; -import org.junit.Ignore; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.List; /** Tests for {@link OceanBaseMySQLCatalogTest}. */ -@Ignore -public class OceanBaseMySQLCatalogTest { +// @Ignore +public class OceanBaseMySQLCatalogTest extends OceanBaseTestBase { private static final ImmutableMap configMap = ImmutableMap.builder() - .put("url", System.getenv("url")) - .put("username", System.getenv("username")) - .put("password", System.getenv("password")) + .put("url", OB_SERVER.getJdbcUrl()) + .put("username", OB_SERVER.getUsername()) + .put("password", OB_SERVER.getPassword()) .build(); @Test diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java new file mode 100644 index 00000000000..eb8e25d35ea --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java @@ -0,0 +1,55 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.factories.DataSinkFactory; +import org.apache.flink.cdc.common.factories.FactoryHelper; +import org.apache.flink.cdc.common.sink.DataSink; +import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; +import org.apache.flink.cdc.connectors.oceanbase.factory.OceanBaseDataSinkFactory; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** Tests for {@link org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory}. */ +public class OceanBaseDataSinkFactoryTest /*extends OceanBaseTestBase*/ { + + @Test + public void testCreateDataSink() { + DataSinkFactory sinkFactory = + FactoryDiscoveryUtils.getFactoryByIdentifier("oceanbase", DataSinkFactory.class); + assertTrue(sinkFactory instanceof OceanBaseDataSinkFactory); + + Configuration conf = + Configuration.fromMap( + ImmutableMap.builder() + .put("url", System.getenv("url")) + .put("username", System.getenv("username")) + .put("password", System.getenv("password")) + .build()); + DataSink dataSink = + sinkFactory.createDataSink( + new FactoryHelper.DefaultContext( + conf, conf, Thread.currentThread().getContextClassLoader())); + assertTrue(dataSink instanceof OceanBaseDataSink); + } +} From 15067d2a3b48306ea00bc2362fdc0877398ceb19 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Thu, 23 May 2024 18:23:56 +0800 Subject: [PATCH 04/23] Enhancement: add mysql to oceanbase end-2-end integrate test --- .../pom.xml | 35 ++- .../oceanbase/catalog/OceanBaseCatalog.java | 29 +- .../catalog/OceanBaseMySQLCatalog.java | 40 ++- .../catalog/OceanBaseOracleCatalog.java | 243 +++------------- .../oceanbase/catalog/OceanBaseTable.java | 65 +---- .../factory/OceanBaseDataSinkFactory.java | 13 +- .../oceanbase/sink/OceanBaseDataSink.java | 2 +- .../OceanBaseEventSerializationSchema.java | 6 +- .../sink/OceanBaseMetadataApplier.java | 6 +- .../oceanbase/sink/OceanBaseUtils.java | 155 ++-------- .../oceanbase/OceanBaseContainer.java | 11 + .../oceanbase/OceanBaseTestBase.java | 1 - .../catalog/OceanBaseMySQLCatalogTest.java | 2 +- .../sink/OceanBaseDataSinkFactoryTest.java | 2 +- .../flink-cdc-pipeline-e2e-tests/pom.xml | 17 ++ .../pipeline/tests/OceanBaseE2eITCase.java | 264 ++++++++++++++++++ .../resources/ddl/mysql_2_oceanbase_test.sql | 21 ++ 17 files changed, 467 insertions(+), 445 deletions(-) create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java create mode 100644 flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index db514dbba15..926bdd63b15 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -22,26 +22,30 @@ limitations under the License. org.apache.flink flink-cdc-pipeline-connectors - 3.2-SNAPSHOT + ${revision} flink-cdc-pipeline-connector-oceanbase + flink-cdc-pipeline-connector-oceanbase + 1.2-SNAPSHOT com.oceanbase flink-connector-oceanbase - 1.1 + ${flink.connector.oceanbase} com.google.guava guava + compile + com.oceanbase oceanbase-client @@ -61,4 +65,31 @@ limitations under the License. test + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + com.oceanbase:* + com.alibaba:* + + + + + + + + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java index d67a06a693a..97a98fb8a84 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java @@ -21,6 +21,7 @@ import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; import org.apache.commons.compress.utils.Lists; +import java.io.Serializable; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -28,10 +29,11 @@ import java.sql.Statement; import java.util.List; import java.util.Objects; +import java.util.Optional; /** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution. */ -public abstract class OceanBaseCatalog { - +public abstract class OceanBaseCatalog implements Serializable { + private static final long serialVersionUID = 1L; private final OceanBaseConnectionProvider connectionProvider; public OceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) { @@ -59,4 +61,27 @@ protected void executeUpdateStatement(String sql) throws SQLException { statement.executeUpdate(sql); } } + + public abstract boolean databaseExists(String databaseName) throws OceanBaseCatalogException; + + public abstract void createDatabase(String databaseName, boolean ignoreIfExists) + throws OceanBaseCatalogException; + + public abstract void createTable(OceanBaseTable table, boolean ignoreIfExists) + throws OceanBaseCatalogException; + + public abstract void alterAddColumns( + String databaseName, String tableName, List addColumns); + + protected abstract String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists); + + protected abstract String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists); + + protected abstract String buildColumnStmt(OceanBaseColumn column); + + protected abstract String getFullColumnType( + String type, Optional columnSize, Optional decimalDigits); + + protected abstract String buildAlterAddColumnsSql( + String databaseName, String tableName, List addColumns); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index 566b623c16c..c446473cfca 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -47,6 +47,7 @@ public OceanBaseMySQLCatalog(OceanBaseConnectorOptions connectorOptions) { * @return true if the given database exists in the catalog false otherwise * @throws OceanBaseCatalogException in case of any runtime exception */ + @Override public boolean databaseExists(String databaseName) throws OceanBaseCatalogException { Preconditions.checkArgument( !StringUtils.isNullOrWhitespaceOnly(databaseName), @@ -77,6 +78,7 @@ public boolean databaseExists(String databaseName) throws OceanBaseCatalogExcept * exists: if set to false, throw a StarRocksCatalogException, if set to true, do nothing. * @throws OceanBaseCatalogException in case of any runtime exception */ + @Override public void createDatabase(String databaseName, boolean ignoreIfExists) throws OceanBaseCatalogException { Preconditions.checkArgument( @@ -104,6 +106,7 @@ public void createDatabase(String databaseName, boolean ignoreIfExists) * it throws a TableAlreadyExistException, if set to true, do nothing. * @throws OceanBaseCatalogException in case of any runtime exception */ + @Override public void createTable(OceanBaseTable table, boolean ignoreIfExists) throws OceanBaseCatalogException { String createTableSql = buildCreateTableSql(table, ignoreIfExists); @@ -129,6 +132,7 @@ public void createTable(OceanBaseTable table, boolean ignoreIfExists) } } + @Override public void alterAddColumns( String databaseName, String tableName, List addColumns) { Preconditions.checkArgument( @@ -165,12 +169,14 @@ public void alterAddColumns( // OceanBase DDL SQL // ------------------------------------------------------------------------------------------ - private String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + @Override + protected String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { return String.format( "CREATE DATABASE %s%s;", ignoreIfExists ? "IF NOT EXISTS " : "", databaseName); } - private String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { + @Override + protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { StringBuilder builder = new StringBuilder(); builder.append( String.format( @@ -184,32 +190,15 @@ private String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) .map(this::buildColumnStmt) .collect(Collectors.joining(",\n")); builder.append(columnsStmt); - builder.append("\n) "); + builder.append(",\n"); - Preconditions.checkArgument( - table.getTableType() == OceanBaseTable.TableType.PRIMARY_KEY, - "Not support to build create table sql for table type " + table.getTableType()); - Preconditions.checkArgument( - table.getTableKeys().isPresent(), - "Can't build create table sql because there is no table keys"); String tableKeys = table.getTableKeys().get().stream() .map(key -> "`" + key + "`") .collect(Collectors.joining(", ")); - builder.append(String.format("PRIMARY KEY (%s)\n", tableKeys)); + builder.append(String.format("PRIMARY KEY (%s)", tableKeys)); + builder.append("\n) "); - Preconditions.checkArgument( - table.getDistributionKeys().isPresent(), - "Can't build create table sql because there is no distribution keys"); - String distributionKeys = - table.getDistributionKeys().get().stream() - .map(key -> "`" + key + "`") - .collect(Collectors.joining(", ")); - builder.append(String.format("DISTRIBUTED BY HASH (%s)", distributionKeys)); - if (table.getNumBuckets().isPresent()) { - builder.append(" BUCKETS "); - builder.append(table.getNumBuckets().get()); - } if (!table.getProperties().isEmpty()) { builder.append("\nPROPERTIES (\n"); String properties = @@ -258,7 +247,8 @@ private String buildAlterDropColumnsSql( return builder.toString(); } - private String buildColumnStmt(OceanBaseColumn column) { + @Override + protected String buildColumnStmt(OceanBaseColumn column) { StringBuilder builder = new StringBuilder(); builder.append("`"); builder.append(column.getColumnName()); @@ -278,7 +268,8 @@ private String buildColumnStmt(OceanBaseColumn column) { return builder.toString(); } - private String getFullColumnType( + @Override + protected String getFullColumnType( String type, Optional columnSize, Optional decimalDigits) { String dataType = type.toUpperCase(); switch (dataType) { @@ -298,6 +289,7 @@ private String getFullColumnType( } } + @Override protected String buildAlterAddColumnsSql( String databaseName, String tableName, List addColumns) { StringBuilder builder = new StringBuilder(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java index 1fb1bfd0f8e..5f359f4e6c3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java @@ -17,16 +17,12 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; - import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; /** * A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under Oracle @@ -40,229 +36,62 @@ public OceanBaseOracleCatalog(OceanBaseConnectorOptions connectorOptions) { super(connectorOptions); } - /** - * Check if a database exists in this catalog. - * - * @param databaseName Name of the database - * @return true if the given database exists in the catalog false otherwise - * @throws OceanBaseCatalogException in case of any runtime exception - */ + @Override public boolean databaseExists(String databaseName) throws OceanBaseCatalogException { - Preconditions.checkArgument( - !StringUtils.isNullOrWhitespaceOnly(databaseName), - "database name cannot be null or empty."); - String querySql = - String.format( - "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA` WHERE SCHEMA_NAME = '%s';", - databaseName); - try { - List dbList = executeSingleColumnStatement(querySql); - return !dbList.isEmpty(); - } catch (Exception e) { - LOG.error( - "Failed to check database exist, database: {}, sql: {}", - databaseName, - querySql, - e); - throw new OceanBaseCatalogException( - String.format("Failed to check database exist, database: %s", databaseName), e); - } + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - /** - * Create a database. - * - * @param databaseName Name of the database - * @param ignoreIfExists Flag to specify behavior when a database with the given name already - * exists: if set to false, throw a StarRocksCatalogException, if set to true, do nothing. - * @throws OceanBaseCatalogException in case of any runtime exception - */ + @Override public void createDatabase(String databaseName, boolean ignoreIfExists) throws OceanBaseCatalogException { - Preconditions.checkArgument( - !StringUtils.isNullOrWhitespaceOnly(databaseName), - "database name cannot be null or empty."); - String sql = buildCreateDatabaseSql(databaseName, ignoreIfExists); - try { - executeUpdateStatement(sql); - LOG.info("Successful to create database {}, sql: {}", databaseName, sql); - } catch (Exception e) { - LOG.info("Failed to create database {}, sql: {}", databaseName, sql, e); - throw new OceanBaseCatalogException( - String.format( - "Failed to create database %s, ignoreIfExists: %s", - databaseName, ignoreIfExists), - e); - } + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - /** - * Creates a table. - * - * @param table the table definition - * @param ignoreIfExists flag to specify behavior when a table already exists. if set to false, - * it throws a TableAlreadyExistException, if set to true, do nothing. - * @throws OceanBaseCatalogException in case of any runtime exception - */ + @Override public void createTable(OceanBaseTable table, boolean ignoreIfExists) throws OceanBaseCatalogException { - String createTableSql = buildCreateTableSql(table, ignoreIfExists); - try { - executeUpdateStatement(createTableSql); - LOG.info( - "Success to create table {}.{}, sql: {}", - table.getDatabaseName(), - table.getDatabaseName(), - createTableSql); - } catch (Exception e) { - LOG.error( - "Failed to create table {}.{}, sql: {}", - table.getDatabaseName(), - table.getDatabaseName(), - createTableSql, - e); - throw new OceanBaseCatalogException( - String.format( - "Failed to create table %s.%s", - table.getDatabaseName(), table.getDatabaseName()), - e); - } + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - // ------------------------------------------------------------------------------------------ - // OceanBase DDL SQL - // ------------------------------------------------------------------------------------------ - - private String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { - return String.format( - "CREATE DATABASE %s%s;", ignoreIfExists ? "IF NOT EXISTS " : "", databaseName); + @Override + public void alterAddColumns( + String databaseName, String tableName, List addColumns) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - private String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { - StringBuilder builder = new StringBuilder(); - builder.append( - String.format( - "CREATE TABLE %s`%s`.`%s`", - ignoreIfExists ? "IF NOT EXISTS " : "", - table.getDatabaseName(), - table.getTableName())); - builder.append(" (\n"); - String columnsStmt = - table.getColumns().stream() - .map(this::buildColumnStmt) - .collect(Collectors.joining(",\n")); - builder.append(columnsStmt); - builder.append("\n) "); - - Preconditions.checkArgument( - table.getTableType() == OceanBaseTable.TableType.PRIMARY_KEY, - "Not support to build create table sql for table type " + table.getTableType()); - Preconditions.checkArgument( - table.getTableKeys().isPresent(), - "Can't build create table sql because there is no table keys"); - String tableKeys = - table.getTableKeys().get().stream() - .map(key -> "`" + key + "`") - .collect(Collectors.joining(", ")); - builder.append(String.format("PRIMARY KEY (%s)\n", tableKeys)); - - Preconditions.checkArgument( - table.getDistributionKeys().isPresent(), - "Can't build create table sql because there is no distribution keys"); - String distributionKeys = - table.getDistributionKeys().get().stream() - .map(key -> "`" + key + "`") - .collect(Collectors.joining(", ")); - builder.append(String.format("DISTRIBUTED BY HASH (%s)", distributionKeys)); - if (table.getNumBuckets().isPresent()) { - builder.append(" BUCKETS "); - builder.append(table.getNumBuckets().get()); - } - if (!table.getProperties().isEmpty()) { - builder.append("\nPROPERTIES (\n"); - String properties = - table.getProperties().entrySet().stream() - .map( - entry -> - String.format( - "\"%s\" = \"%s\"", - entry.getKey(), entry.getValue())) - .collect(Collectors.joining(",\n")); - builder.append(properties); - builder.append("\n)"); - } - builder.append(";"); - return builder.toString(); + @Override + protected String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - private String buildAlterAddColumnsSql( - String databaseName, - String tableName, - List addColumns, - long timeoutSecond) { - StringBuilder builder = new StringBuilder(); - builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); - String columnsStmt = - addColumns.stream() - .map(col -> "ADD COLUMN " + buildColumnStmt(col)) - .collect(Collectors.joining(", ")); - builder.append(columnsStmt); - builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); - builder.append(";"); - return builder.toString(); + @Override + protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - private String buildAlterDropColumnsSql( - String databaseName, String tableName, List dropColumns, long timeoutSecond) { - StringBuilder builder = new StringBuilder(); - builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); - String columnsStmt = - dropColumns.stream() - .map(col -> String.format("DROP COLUMN `%s`", col)) - .collect(Collectors.joining(", ")); - builder.append(columnsStmt); - builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); - builder.append(";"); - return builder.toString(); + @Override + protected String buildColumnStmt(OceanBaseColumn column) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - private String buildColumnStmt(OceanBaseColumn column) { - StringBuilder builder = new StringBuilder(); - builder.append("`"); - builder.append(column.getColumnName()); - builder.append("` "); - builder.append( - getFullColumnType( - column.getDataType(), column.getColumnSize(), column.getDecimalDigits())); - builder.append(" "); - builder.append(column.isNullable() ? "NULL" : "NOT NULL"); - if (column.getDefaultValue().isPresent()) { - builder.append(String.format(" DEFAULT \"%s\"", column.getDefaultValue().get())); - } - - if (column.getColumnComment().isPresent()) { - builder.append(String.format(" COMMENT \"%s\"", column.getColumnComment().get())); - } - return builder.toString(); + @Override + protected String getFullColumnType( + String type, Optional columnSize, Optional decimalDigits) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } - private String getFullColumnType( - String type, Optional columnSize, Optional decimalDigits) { - String dataType = type.toUpperCase(); - switch (dataType) { - case "DECIMAL": - Preconditions.checkArgument( - columnSize.isPresent(), "DECIMAL type must have column size"); - Preconditions.checkArgument( - decimalDigits.isPresent(), "DECIMAL type must have decimal digits"); - return String.format("DECIMAL(%d, %s)", columnSize.get(), decimalDigits.get()); - case "CHAR": - case "VARCHAR": - Preconditions.checkArgument( - columnSize.isPresent(), type + " type must have column size"); - return String.format("%s(%d)", dataType, columnSize.get()); - default: - return dataType; - } + @Override + protected String buildAlterAddColumnsSql( + String databaseName, String tableName, List addColumns) { + throw new OceanBaseCatalogException( + "This operator under oracle tenant is not supported currently."); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java index 4cf2f39a320..9f8080ade6c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java @@ -28,22 +28,13 @@ import java.util.Objects; import java.util.Optional; -/** - * Describe a OceanBase table. See OceanBase table - * design for how to define a OceanBase table. - */ +/** Describe a OceanBase table. */ public class OceanBaseTable { - /** - * Types of OceanBase table. See OceanBase Table Types. - */ + /** Types of OceanBase table. */ public enum TableType { UNKNOWN, DUPLICATE_KEY, - AGGREGATE, - UNIQUE_KEY, PRIMARY_KEY } @@ -59,20 +50,8 @@ public enum TableType { /** The columns sorted by the ordinal position. */ private final List columns; - /** - * The table keys sorted by the ordinal position. null if it's unknown. The table keys has - * different meaning for different types of tables. For duplicate key table, It's duplicate - * keys. For aggregate table, it's aggregate keys. For unique key table, it's unique keys. For - * primary key table, it's primary keys. - */ @Nullable private final List tableKeys; - /** The distribution keys. null if it's unknown. */ - @Nullable private final List distributionKeys; - - /** The number of buckets. null if it's unknown or automatic. */ - @Nullable private final Integer numBuckets; - /** The table comment. null if there is no comment or it's unknown. */ @Nullable private final String comment; @@ -88,8 +67,6 @@ private OceanBaseTable( TableType tableType, List columns, @Nullable List tableKeys, - @Nullable List distributionKeys, - @Nullable Integer numBuckets, @Nullable String comment, Map properties) { Preconditions.checkNotNull(databaseName); @@ -101,8 +78,6 @@ private OceanBaseTable( this.tableType = tableType; this.columns = columns; this.tableKeys = tableKeys; - this.distributionKeys = distributionKeys; - this.numBuckets = numBuckets; this.comment = comment; this.properties = Preconditions.checkNotNull(properties); } @@ -127,14 +102,6 @@ public Optional> getTableKeys() { return Optional.ofNullable(tableKeys); } - public Optional> getDistributionKeys() { - return Optional.ofNullable(distributionKeys); - } - - public Optional getNumBuckets() { - return Optional.ofNullable(numBuckets); - } - public Optional getComment() { return Optional.ofNullable(comment); } @@ -172,10 +139,6 @@ public String toString() { + columns + ", tableKeys=" + tableKeys - + ", distributionKeys=" - + distributionKeys - + ", numBuckets=" - + numBuckets + ", comment='" + comment + '\'' @@ -198,8 +161,6 @@ public boolean equals(Object o) { && tableType == that.tableType && Objects.equals(columns, that.columns) && Objects.equals(tableKeys, that.tableKeys) - && Objects.equals(distributionKeys, that.distributionKeys) - && Objects.equals(numBuckets, that.numBuckets) && Objects.equals(comment, that.comment) && Objects.equals(properties, that.properties); } @@ -212,8 +173,6 @@ public static class Builder { private TableType tableType; private List columns = new ArrayList<>(); private List tableKeys; - private List distributionKeys; - private Integer numBuckets; private String comment; private Map properties = new HashMap<>(); @@ -242,16 +201,6 @@ public Builder setTableKeys(List tableKeys) { return this; } - public Builder setDistributionKeys(List distributionKeys) { - this.distributionKeys = distributionKeys; - return this; - } - - public Builder setNumBuckets(Integer numBuckets) { - this.numBuckets = numBuckets; - return this; - } - public Builder setComment(String comment) { this.comment = comment; return this; @@ -264,15 +213,7 @@ public Builder setTableProperties(Map properties) { public OceanBaseTable build() { return new OceanBaseTable( - databaseName, - tableName, - tableType, - columns, - tableKeys, - distributionKeys, - numBuckets, - comment, - properties); + databaseName, tableName, tableType, columns, tableKeys, comment, properties); } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index 83339cae2d3..bbd6c248ed0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -29,6 +29,8 @@ import java.time.ZoneId; import java.util.HashSet; +import java.util.Map; +import java.util.Optional; import java.util.Set; import static org.apache.flink.cdc.common.pipeline.PipelineOptions.PIPELINE_LOCAL_TIME_ZONE; @@ -40,7 +42,8 @@ public class OceanBaseDataSinkFactory implements DataSinkFactory { @Override public DataSink createDataSink(Context context) { Configuration config = context.getFactoryConfiguration(); - OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(config.toMap()); + OceanBaseConnectorOptions connectorOptions = + new OceanBaseConnectorOptions(buildOceanBaseOptions(config)); String zoneStr = context.getFactoryConfiguration().get(PIPELINE_LOCAL_TIME_ZONE); ZoneId zoneId = PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(zoneStr) @@ -49,6 +52,14 @@ public DataSink createDataSink(Context context) { return new OceanBaseDataSink(connectorOptions, config, zoneId); } + public Map buildOceanBaseOptions(Configuration config) { + Optional optional = config.getOptional(OceanBaseDataSinkOptions.PASSWORD); + config.remove(OceanBaseDataSinkOptions.PASSWORD); + Map map = config.toMap(); + map.put(OceanBaseDataSinkOptions.PASSWORD.key(), optional.orElse("")); + return map; + } + @Override public String identifier() { return "oceanbase"; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java index 49da92c6af7..3605bd75c43 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -68,7 +68,7 @@ public MetadataApplier getMetadataApplier() { try { return new OceanBaseMetadataApplier(connectorOptions, config); } catch (Exception e) { - throw new RuntimeException("Fail to create metadata applier"); + throw new RuntimeException(e.getMessage(), e); } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index 969ac4cea4b..b8d3c0c69e0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -43,6 +43,7 @@ /** A serializer for Event to Record. */ public class OceanBaseEventSerializationSchema implements RecordSerializationSchema { + private Map schemaMaps = new HashMap<>(); /** ZoneId from pipeline config to support timestamp with local time zone. */ @@ -94,7 +95,6 @@ private Record applyDataChangeEvent(DataChangeEvent event) { default: throw new UnsupportedOperationException("Unsupport Operation " + op); } - return buildDataChangeRecord(tableId, schema, values, isDelete); } @@ -106,7 +106,7 @@ private DataChangeRecord buildDataChangeRecord( TableInfo tableInfo = new TableInfo( oceanBaseTableId, - schema.partitionKeys(), + schema.primaryKeys(), schema.getColumnNames(), Lists.newArrayList(), null); @@ -117,7 +117,7 @@ private DataChangeRecord buildDataChangeRecord( values); } - /** serializer RecordData to Doris Value. */ + /** serializer RecordData to oceanbase data change record. */ public Object[] serializerRecord(RecordData recordData, Schema schema) { List columns = schema.getColumns(); Preconditions.checkState( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index fd72c8997db..91e60ad80d5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -125,14 +125,12 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { } private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { - // TODO OceanBase plans to support column drop since 3.3 which has not been released. - // Support it later. + // TODO throw new UnsupportedOperationException("Rename column is not supported currently"); } private void applyRenameColumnEvent(RenameColumnEvent renameColumnEvent) { - // TODO OceanBase plans to support column rename since 3.3 which has not been released. - // Support it later. + // TODO throw new UnsupportedOperationException("Rename column is not supported currently"); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index 58255d5b28a..26c2007a994 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; -import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; @@ -39,31 +38,17 @@ import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; -import java.time.LocalDate; -import java.time.ZoneId; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; +import org.apache.commons.collections.CollectionUtils; + import java.util.ArrayList; import java.util.List; -import static org.apache.flink.cdc.common.types.DataTypeChecks.getPrecision; -import static org.apache.flink.cdc.common.types.DataTypeChecks.getScale; - /** Utilities for conversion from source table to OceanBase table. */ public class OceanBaseUtils { /** Convert a source table to {@link OceanBaseTable}. */ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { - if (schema.primaryKeys().isEmpty()) { - throw new RuntimeException( - String.format( - "Only support OceanBase primary key table, but the source table %s has no primary keys", - tableId)); - } - // For OceanBase primary key table DDL, primary key columns must be defined before other - // columns, - // so reorder the columns in the source schema to make primary key columns at the front List orderedColumns = new ArrayList<>(); for (String primaryKey : schema.primaryKeys()) { orderedColumns.add(schema.getColumn(primaryKey).get()); @@ -91,11 +76,12 @@ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { new OceanBaseTable.Builder() .setDatabaseName(tableId.getSchemaName()) .setTableName(tableId.getTableName()) - .setTableType(OceanBaseTable.TableType.PRIMARY_KEY) + .setTableType( + CollectionUtils.isNotEmpty(schema.primaryKeys()) + ? OceanBaseTable.TableType.PRIMARY_KEY + : OceanBaseTable.TableType.DUPLICATE_KEY) .setColumns(oceanBaseColumns) .setTableKeys(schema.primaryKeys()) - // use primary keys as distribution keys by default - .setDistributionKeys(schema.primaryKeys()) .setComment(schema.comment()); return tableBuilder.build(); } @@ -108,100 +94,6 @@ public static void toOceanBaseDataType( cdcColumn.getType().accept(dataTypeTransformer); } - /** Format DATE type data. */ - private static final DateTimeFormatter DATE_FORMATTER = - DateTimeFormatter.ofPattern("yyyy-MM-dd"); - - /** Format timestamp-related type data. */ - private static final DateTimeFormatter DATETIME_FORMATTER = - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - - /** - * Creates an accessor for getting elements in an internal RecordData structure at the given - * position. - * - * @param fieldType the element type of the RecordData - * @param fieldPos the element position of the RecordData - * @param zoneId the time zone used when converting from TIMESTAMP WITH LOCAL TIME ZONE - * - */ - public static RecordData.FieldGetter createFieldGetter( - DataType fieldType, int fieldPos, ZoneId zoneId) { - final RecordData.FieldGetter fieldGetter; - // ordered by type root definition - switch (fieldType.getTypeRoot()) { - case BOOLEAN: - fieldGetter = record -> record.getBoolean(fieldPos); - break; - case TINYINT: - fieldGetter = record -> record.getByte(fieldPos); - break; - case SMALLINT: - fieldGetter = record -> record.getShort(fieldPos); - break; - case INTEGER: - fieldGetter = record -> record.getInt(fieldPos); - break; - case BIGINT: - fieldGetter = record -> record.getLong(fieldPos); - break; - case FLOAT: - fieldGetter = record -> record.getFloat(fieldPos); - break; - case DOUBLE: - fieldGetter = record -> record.getDouble(fieldPos); - break; - case DECIMAL: - final int decimalPrecision = getPrecision(fieldType); - final int decimalScale = getScale(fieldType); - fieldGetter = - record -> - record.getDecimal(fieldPos, decimalPrecision, decimalScale) - .toBigDecimal(); - break; - case CHAR: - case VARCHAR: - fieldGetter = record -> record.getString(fieldPos).toString(); - break; - case DATE: - fieldGetter = - record -> - LocalDate.ofEpochDay(record.getInt(fieldPos)) - .format(DATE_FORMATTER); - break; - case TIMESTAMP_WITHOUT_TIME_ZONE: - fieldGetter = - record -> - record.getTimestamp(fieldPos, getPrecision(fieldType)) - .toLocalDateTime() - .format(DATETIME_FORMATTER); - break; - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - fieldGetter = - record -> - ZonedDateTime.ofInstant( - record.getLocalZonedTimestampData( - fieldPos, getPrecision(fieldType)) - .toInstant(), - zoneId) - .toLocalDateTime() - .format(DATETIME_FORMATTER); - break; - default: - throw new UnsupportedOperationException( - "Don't support data type " + fieldType.getTypeRoot()); - } - if (!fieldType.isNullable()) { - return fieldGetter; - } - return row -> { - if (row.isNullAt(fieldPos)) { - return null; - } - return fieldGetter.getFieldOrNull(row); - }; - } - // ------------------------------------------------------------------------------------------ // OceanBase data types // ------------------------------------------------------------------------------------------ @@ -221,12 +113,14 @@ fieldPos, getPrecision(fieldType)) public static final String DATE = "DATE"; public static final String DATETIME = "DATETIME"; public static final String JSON = "JSON"; + public static final String TEXT = "TEXT"; + public static final String BLOB = "BLOB"; /** Max size of char type of OceanBase. */ - public static final int MAX_CHAR_SIZE = 255; + public static final int MAX_CHAR_SIZE = 256; /** Max size of varchar type of OceanBase. */ - public static final int MAX_VARCHAR_SIZE = 1048576; + public static final int MAX_VARCHAR_SIZE = 262144; /** Transforms CDC {@link DataType} to OceanBase data type. */ public static class CdcDataTypeTransformer @@ -315,41 +209,30 @@ public OceanBaseColumn.Builder visit(DecimalType decimalType) { @Override public OceanBaseColumn.Builder visit(CharType charType) { - // CDC and OceanBase use different units for the length. It's the number - // of characters in CDC, and the number of bytes in OceanBase. One chinese - // character will use 3 bytes because it uses UTF-8, so the length of OceanBase - // char type should be three times as that of CDC char type. Specifically, if - // the length of OceanBase exceeds the MAX_CHAR_SIZE, map CDC char type to OceanBase - // varchar type int length = charType.getLength(); - long oceanBaseLength = length * 3L; - // In the OceanBase, The primary key columns can be any of the following data types: - // BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, STRING, VARCHAR, DATE, and - // DATETIME, But it doesn't include CHAR. When a char type appears in the primary key of - // MySQL, creating a table in OceanBase requires conversion to varchar type. - if (oceanBaseLength <= MAX_CHAR_SIZE && !isPrimaryKeys) { + if (length <= MAX_CHAR_SIZE) { builder.setDataType(CHAR); builder.setNullable(charType.isNullable()); - builder.setColumnSize((int) oceanBaseLength); + builder.setColumnSize(length); } else { builder.setDataType(VARCHAR); builder.setNullable(charType.isNullable()); - builder.setColumnSize((int) Math.min(oceanBaseLength, MAX_VARCHAR_SIZE)); + builder.setColumnSize(Math.min(length, MAX_VARCHAR_SIZE)); } return builder; } @Override public OceanBaseColumn.Builder visit(VarCharType varCharType) { - // CDC and OceanBase use different units for the length. It's the number - // of characters in CDC, and the number of bytes in OceanBase. One chinese - // character will use 3 bytes because it uses UTF-8, so the length of OceanBase - // varchar type should be three times as that of CDC varchar type. int length = varCharType.getLength(); - long oceanBaseLength = length * 3L; builder.setDataType(VARCHAR); builder.setNullable(varCharType.isNullable()); - builder.setColumnSize((int) Math.min(oceanBaseLength, MAX_VARCHAR_SIZE)); + builder.setColumnSize(Math.min(length, MAX_VARCHAR_SIZE)); + + // case for string type to avoid row size too large + if (varCharType.getLength() == VarCharType.MAX_LENGTH) { + builder.setDataType(TEXT); + } return builder; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java index a952853ac8a..548e36e6bac 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java @@ -85,6 +85,17 @@ public String getJdbcUrl(String databaseName) { + additionalUrlParams; } + public String getJdbcUrlInContainer(String databaseName, String networkAliases) { + String additionalUrlParams = constructUrlParameters("?", "&"); + return "jdbc:mysql://" + + networkAliases + + ":" + + SQL_PORT + + "/" + + databaseName + + additionalUrlParams; + } + public OceanBaseContainer withSysPassword(String sysPassword) { this.sysPassword = sysPassword; return this; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java index eb95c50a088..73109d74214 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java @@ -36,7 +36,6 @@ public class OceanBaseTestBase { @ClassRule public static final OceanBaseContainer OB_SERVER = new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) - .withNetworkMode("host") .withSysPassword("123456") .withCopyFileToContainer( MountableFile.forClasspathResource("sql/init.sql"), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java index 54967deee7c..86e39f820c8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -23,8 +23,8 @@ import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.junit.Test; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; import java.util.List; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java index eb8e25d35ea..2eb7eae33d5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java @@ -31,7 +31,7 @@ import static org.junit.Assert.assertTrue; /** Tests for {@link org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory}. */ -public class OceanBaseDataSinkFactoryTest /*extends OceanBaseTestBase*/ { +public class OceanBaseDataSinkFactoryTest { @Test public void testCreateDataSink() { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml index e785ba0a5a8..3b855a5e928 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml @@ -110,6 +110,13 @@ limitations under the License. ${testcontainers.version} test + + org.apache.flink + flink-cdc-pipeline-connector-oceanbase + ${project.version} + test-jar + test + @@ -228,6 +235,16 @@ limitations under the License. ${project.build.directory}/dependencies + + + org.apache.flink + flink-cdc-pipeline-connector-oceanbase + ${project.version} + oceanbase-cdc-pipeline-connector.jar + jar + ${project.build.directory}/dependencies + + diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java new file mode 100644 index 00000000000..a0153be1845 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -0,0 +1,264 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.pipeline.tests; + +import org.apache.flink.cdc.common.test.utils.TestUtils; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.connectors.oceanbase.OceanBaseContainer; +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; + +import com.oceanbase.connector.flink.utils.OceanBaseJdbcUtils; +import org.apache.commons.text.StringEscapeUtils; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; + +import java.nio.file.Path; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase.IMAGE_TAG; + +/** OceanBase flink cdc pipeline connector sink integrate test. */ +@RunWith(Parameterized.class) +public class OceanBaseE2eITCase extends PipelineTestEnvironment { + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseE2eITCase.class); + + // ------------------------------------------------------------------------------------------ + // MySQL Variables (we always use MySQL as the data source for easier verifying) + // ------------------------------------------------------------------------------------------ + protected static final String MYSQL_TEST_USER = "mysqluser"; + protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; + protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; + + private static final String MYSQL_TEST_TABLE_NAME = "products"; + + @ClassRule + public static final MySqlContainer MYSQL = + (MySqlContainer) + new MySqlContainer( + MySqlVersion.V8_0) // v8 support both ARM and AMD architectures + .withConfigurationOverride("docker/mysql/my.cnf") + .withSetupSQL("docker/mysql/setup.sql") + .withDatabaseName("flink-test") + .withUsername("flinkuser") + .withPassword("flinkpw") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + public final UniqueDatabase mysqlInventoryDatabase = + new UniqueDatabase( + MYSQL, "mysql_2_oceanbase_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + + private final String uniqueDatabaseName = mysqlInventoryDatabase.getDatabaseName(); + + @Rule + public final OceanBaseContainer obServer = + new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) + .withNetwork(NETWORK) + .withNetworkAliases("oceanbase") + .withSysPassword("123456") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @Before + public void before() throws Exception { + super.before(); + mysqlInventoryDatabase.createAndInitialize(); + } + + @After + public void after() { + super.after(); + mysqlInventoryDatabase.dropDatabase(); + } + + @Test + public void testSyncWholeDatabase() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n\n" + + "sink:\n" + + " type: oceanbase\n" + + " url: %s\n" + + " username: %s\n" + + " password: %s\n" + + "\n" + + "pipeline:\n" + + " name: oceanbase IT\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + uniqueDatabaseName, + obServer.getJdbcUrlInContainer("test", "oceanbase"), + obServer.getUsername(), + obServer.getPassword()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path oceanbaseCdcJar = TestUtils.getResource("oceanbase-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, oceanbaseCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + // waiting for databases were created in oceanbase + Thread.sleep(30_000); + + waitForTableCount(MYSQL_TEST_TABLE_NAME, false, 9); + List originList = queryTable(MYSQL_TEST_TABLE_NAME, false); + MatcherAssert.assertThat( + originList, + Matchers.containsInAnyOrder( + Stream.of( + "101,scooter,Small 2-wheel scooter,3.14,red,{\"key1\": \"value1\"},{\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}", + "102,car battery,12V car battery,8.1,white,{\"key2\": \"value2\"},{\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0}", + "103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8,red,{\"key3\": \"value3\"},{\"coordinates\":[3,3],\"type\":\"Point\",\"srid\":0}", + "104,hammer,12oz carpenter's hammer,0.75,white,{\"key4\": \"value4\"},{\"coordinates\":[4,4],\"type\":\"Point\",\"srid\":0}", + "105,hammer,14oz carpenter's hammer,0.875,red,{\"k1\": \"v1\", \"k2\": \"v2\"},{\"coordinates\":[5,5],\"type\":\"Point\",\"srid\":0}", + "106,hammer,16oz carpenter's hammer,1.0,null,null,null", + "107,rocks,box of assorted rocks,5.3,null,null,null", + "108,jacket,water resistent black wind breaker,0.1,null,null,null", + "109,spare tire,24 inch spare tire,22.2,null,null,null") + .map(StringEscapeUtils::unescapeJava) + .toArray())); + + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), MYSQL.getDatabasePort(), uniqueDatabaseName); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); + stat.execute("UPDATE products SET weight='5.1' WHERE id=107;"); + // modify table schema + stat.execute("ALTER TABLE products ADD COLUMN new_col INT;"); + stat.execute( + "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2, null, null, null, 1);"); // 110 + stat.execute( + "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18, null, null, null, 1);"); // 111 + stat.execute( + "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); + stat.execute("UPDATE products SET weight='5.17' WHERE id=111;"); + stat.execute("DELETE FROM products WHERE id=111;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + waitForTableCount(MYSQL_TEST_TABLE_NAME, false, 10); + List updateList = queryTable(MYSQL_TEST_TABLE_NAME, false); + MatcherAssert.assertThat( + updateList, + Matchers.containsInAnyOrder( + Stream.of( + "101,scooter,Small 2-wheel scooter,3.14,red,{\"key1\": \"value1\"},{\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0},null", + "102,car battery,12V car battery,8.1,white,{\"key2\": \"value2\"},{\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0},null", + "103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8,red,{\"key3\": \"value3\"},{\"coordinates\":[3,3],\"type\":\"Point\",\"srid\":0},null", + "104,hammer,12oz carpenter's hammer,0.75,white,{\"key4\": \"value4\"},{\"coordinates\":[4,4],\"type\":\"Point\",\"srid\":0},null", + "105,hammer,14oz carpenter's hammer,0.875,red,{\"k1\": \"v1\", \"k2\": \"v2\"},{\"coordinates\":[5,5],\"type\":\"Point\",\"srid\":0},null", + "106,hammer,18oz carpenter hammer,1.0,null,null,null,null", + "107,rocks,box of assorted rocks,5.1,null,null,null,null", + "108,jacket,water resistent black wind breaker,0.1,null,null,null,null", + "109,spare tire,24 inch spare tire,22.2,null,null,null,null", + "110,jacket,new water resistent white wind breaker,0.5,null,null,null,1") + .map(StringEscapeUtils::unescapeJava) + .toArray())); + } + + private void waitForTableCount(String tableName, boolean isMySQL, int expectedCount) + throws InterruptedException { + while (OceanBaseJdbcUtils.getTableRowsCount( + () -> getConnection(uniqueDatabaseName, isMySQL), tableName) + < expectedCount) { + Thread.sleep(100); + } + } + + private List queryTable(String tableName, boolean isMySQL) throws SQLException { + return queryTable(tableName, isMySQL, Collections.singletonList("*")); + } + + private List queryTable(String tableName, boolean isMySQL, List fields) + throws SQLException { + List result = new ArrayList<>(); + try (Connection connection = getConnection(tableName, isMySQL); + Statement statement = connection.createStatement()) { + ResultSet rs = + statement.executeQuery( + "SELECT " + String.join(", ", fields) + " FROM " + tableName); + ResultSetMetaData metaData = rs.getMetaData(); + + while (rs.next()) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < metaData.getColumnCount(); i++) { + if (i != 0) { + sb.append(","); + } + sb.append(rs.getObject(i + 1)); + } + result.add(sb.toString()); + } + } + return result; + } + + private Connection getConnection(String database, boolean isMySQL) throws SQLException { + if (isMySQL) { + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), MYSQL.getDatabasePort(), uniqueDatabaseName); + System.out.println(mysqlJdbcUrl); + return DriverManager.getConnection(mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + } + return DriverManager.getConnection( + obServer.getJdbcUrl(uniqueDatabaseName), + obServer.getUsername(), + obServer.getPassword()); + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql new file mode 100644 index 00000000000..f60ae607a71 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql @@ -0,0 +1,21 @@ +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + description VARCHAR(512), + weight FLOAT, + enum_c enum('red', 'white') default 'red', -- test some complex types as well, + json_c JSON, -- because we use additional dependencies to deserialize complex types. + point_c POINT +); +ALTER TABLE products AUTO_INCREMENT = 101; + +INSERT INTO products +VALUES (default,"scooter","Small 2-wheel scooter",3.14, 'red', '{"key1": "value1"}', ST_GeomFromText('POINT(1 1)')), + (default,"car battery","12V car battery",8.1, 'white', '{"key2": "value2"}', ST_GeomFromText('POINT(2 2)')), + (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8, 'red', '{"key3": "value3"}', ST_GeomFromText('POINT(3 3)')), + (default,"hammer","12oz carpenter's hammer",0.75, 'white', '{"key4": "value4"}', ST_GeomFromText('POINT(4 4)')), + (default,"hammer","14oz carpenter's hammer",0.875, 'red', '{"k1": "v1", "k2": "v2"}', ST_GeomFromText('POINT(5 5)')), + (default,"hammer","16oz carpenter's hammer",1.0, null, null, null), + (default,"rocks","box of assorted rocks",5.3, null, null, null), + (default,"jacket","water resistent black wind breaker",0.1, null, null, null), + (default,"spare tire","24 inch spare tire",22.2, null, null, null); \ No newline at end of file From a6e73075cae28a46f3dcd3dd487cc8a0adabb39a Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 24 May 2024 14:18:10 +0800 Subject: [PATCH 05/23] Enhancement: refine oceanbase catalog --- .../oceanbase/catalog/OceanBaseCatalog.java | 14 +--- .../oceanbase/catalog/OceanBaseColumn.java | 15 ++-- .../catalog/OceanBaseMySQLCatalog.java | 74 ++++++++++--------- .../catalog/OceanBaseOracleCatalog.java | 40 ++-------- .../OceanBaseEventSerializationSchema.java | 2 +- .../sink/OceanBaseMetadataApplier.java | 29 ++++---- .../oceanbase/sink/OceanBaseUtils.java | 41 ++-------- .../pipeline/tests/OceanBaseE2eITCase.java | 28 ++++--- 8 files changed, 95 insertions(+), 148 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java index 97a98fb8a84..ccdc69e39b6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java @@ -29,7 +29,6 @@ import java.sql.Statement; import java.util.List; import java.util.Objects; -import java.util.Optional; /** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution. */ public abstract class OceanBaseCatalog implements Serializable { @@ -73,15 +72,6 @@ public abstract void createTable(OceanBaseTable table, boolean ignoreIfExists) public abstract void alterAddColumns( String databaseName, String tableName, List addColumns); - protected abstract String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists); - - protected abstract String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists); - - protected abstract String buildColumnStmt(OceanBaseColumn column); - - protected abstract String getFullColumnType( - String type, Optional columnSize, Optional decimalDigits); - - protected abstract String buildAlterAddColumnsSql( - String databaseName, String tableName, List addColumns); + public abstract void renameColumn( + String schemaName, String tableName, String oldColumnName, String newColumnName); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java index 1d4f501a9c8..9c821a3cd64 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java @@ -25,27 +25,24 @@ import static org.apache.flink.util.Preconditions.checkNotNull; -/** Describe a column of OceanBase table. These metas are from information_schema.COLUMNS. */ +/** Describe a column of OceanBase table. */ public class OceanBaseColumn implements Serializable { private static final long serialVersionUID = 1L; - /** The name of the column. COLUMN_NAME in information_schema.COLUMNS. */ + /** The name of the column. */ private final String columnName; - /** - * The position of the column within the table (starting at 0). ORDINAL_POSITION in - * information_schema.COLUMNS. - */ + /** The position of the column within the table (starting at 0). */ private final int ordinalPosition; - /** The column data type. DATA_TYPE in information_schema.COLUMNS. */ + /** The column data type. */ private final String dataType; - /** The column nullability.IS_NULLABLE in information_schema.COLUMNS. */ + /** The column nullability. */ private final boolean isNullable; - /** The default value for the column. COLUMN_DEFAULT in information_schema.COLUMNS. */ + /** The default value for the column. */ @Nullable private final String defaultValue; /** diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index c446473cfca..1f09de9de3b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -34,6 +34,8 @@ */ public class OceanBaseMySQLCatalog extends OceanBaseCatalog { + private static final String RENAME_DDL = "ALTER TABLE `%s`.`%s` RENAME COLUMN `%s` TO `%s`"; + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLCatalog.class); public OceanBaseMySQLCatalog(OceanBaseConnectorOptions connectorOptions) { @@ -165,17 +167,46 @@ public void alterAddColumns( } } + @Override + public void renameColumn( + String schemaName, String tableName, String oldColumnName, String newColumnName) { + String renameColumnSql = + buildRenameColumnSql(schemaName, tableName, oldColumnName, newColumnName); + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(renameColumnSql); + LOG.info( + "Success to rename {} column from {} to {}, duration: {}ms, sql: {}", + String.format("%s.%s", schemaName, tableName), + oldColumnName, + newColumnName, + System.currentTimeMillis() - startTimeMillis, + renameColumnSql); + } catch (Exception e) { + LOG.error( + "Fail to rename {} column from {} to {}, duration: {}ms, sql: {}", + String.format("%s.%s", schemaName, tableName), + oldColumnName, + newColumnName, + renameColumnSql, + e); + throw new OceanBaseCatalogException( + String.format( + "Failed to rename %s column from %s to %s ", + String.format("%s.%s", schemaName, tableName), schemaName, tableName), + e); + } + } + // ------------------------------------------------------------------------------------------ // OceanBase DDL SQL // ------------------------------------------------------------------------------------------ - @Override protected String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { return String.format( "CREATE DATABASE %s%s;", ignoreIfExists ? "IF NOT EXISTS " : "", databaseName); } - @Override protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { StringBuilder builder = new StringBuilder(); builder.append( @@ -216,38 +247,6 @@ protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExist return builder.toString(); } - private String buildAlterAddColumnsSql( - String databaseName, - String tableName, - List addColumns, - long timeoutSecond) { - StringBuilder builder = new StringBuilder(); - builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); - String columnsStmt = - addColumns.stream() - .map(col -> "ADD COLUMN " + buildColumnStmt(col)) - .collect(Collectors.joining(", ")); - builder.append(columnsStmt); - builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); - builder.append(";"); - return builder.toString(); - } - - private String buildAlterDropColumnsSql( - String databaseName, String tableName, List dropColumns, long timeoutSecond) { - StringBuilder builder = new StringBuilder(); - builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); - String columnsStmt = - dropColumns.stream() - .map(col -> String.format("DROP COLUMN `%s`", col)) - .collect(Collectors.joining(", ")); - builder.append(columnsStmt); - builder.append(String.format(" PROPERTIES (\"timeout\" = \"%s\")", timeoutSecond)); - builder.append(";"); - return builder.toString(); - } - - @Override protected String buildColumnStmt(OceanBaseColumn column) { StringBuilder builder = new StringBuilder(); builder.append("`"); @@ -268,7 +267,6 @@ protected String buildColumnStmt(OceanBaseColumn column) { return builder.toString(); } - @Override protected String getFullColumnType( String type, Optional columnSize, Optional decimalDigits) { String dataType = type.toUpperCase(); @@ -289,7 +287,6 @@ protected String getFullColumnType( } } - @Override protected String buildAlterAddColumnsSql( String databaseName, String tableName, List addColumns) { StringBuilder builder = new StringBuilder(); @@ -302,4 +299,9 @@ protected String buildAlterAddColumnsSql( builder.append(";"); return builder.toString(); } + + private static String buildRenameColumnSql( + String schemaName, String tableName, String oldColumnName, String newColumnName) { + return String.format(RENAME_DDL, schemaName, tableName, oldColumnName, newColumnName); + } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java index 5f359f4e6c3..afb6e112e74 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java @@ -22,7 +22,6 @@ import org.slf4j.LoggerFactory; import java.util.List; -import java.util.Optional; /** * A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution under Oracle @@ -39,59 +38,34 @@ public OceanBaseOracleCatalog(OceanBaseConnectorOptions connectorOptions) { @Override public boolean databaseExists(String databaseName) throws OceanBaseCatalogException { throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); + "This operation under oracle tenant is not supported currently."); } @Override public void createDatabase(String databaseName, boolean ignoreIfExists) throws OceanBaseCatalogException { throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); + "This operation under oracle tenant is not supported currently."); } @Override public void createTable(OceanBaseTable table, boolean ignoreIfExists) throws OceanBaseCatalogException { throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); + "This operation under oracle tenant is not supported currently."); } @Override public void alterAddColumns( String databaseName, String tableName, List addColumns) { throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); + "This operation under oracle tenant is not supported currently."); } @Override - protected String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + public void renameColumn( + String schemaName, String tableName, String oldColumnName, String newColumnName) { throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); - } - - @Override - protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExists) { - throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); - } - - @Override - protected String buildColumnStmt(OceanBaseColumn column) { - throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); - } - - @Override - protected String getFullColumnType( - String type, Optional columnSize, Optional decimalDigits) { - throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); - } - - @Override - protected String buildAlterAddColumnsSql( - String databaseName, String tableName, List addColumns) { - throw new OceanBaseCatalogException( - "This operator under oracle tenant is not supported currently."); + "This operation under oracle tenant is not supported currently."); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index b8d3c0c69e0..94c40664f6b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -44,7 +44,7 @@ /** A serializer for Event to Record. */ public class OceanBaseEventSerializationSchema implements RecordSerializationSchema { - private Map schemaMaps = new HashMap<>(); + private final Map schemaMaps = new HashMap<>(); /** ZoneId from pipeline config to support timestamp with local time zone. */ public final ZoneId pipelineZoneId; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 91e60ad80d5..9666f2f8189 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -28,10 +28,10 @@ import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalog; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; -import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseMySQLCatalog; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; @@ -40,23 +40,18 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; /** Supports {@link OceanBaseDataSink} to schema evolution. */ public class OceanBaseMetadataApplier implements MetadataApplier { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMetadataApplier.class); - private final OceanBaseConnectorOptions connectorOptions; - private final Configuration config; - private final OceanBaseMySQLCatalog catalog; + private final OceanBaseCatalog catalog; public OceanBaseMetadataApplier( OceanBaseConnectorOptions connectorOptions, Configuration config) throws Exception { - this.connectorOptions = connectorOptions; - this.config = config; - this.catalog = - (OceanBaseMySQLCatalog) - OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); + this.catalog = OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); } @Override @@ -125,12 +120,20 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { } private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { - // TODO - throw new UnsupportedOperationException("Rename column is not supported currently"); + // TODO The `DropColumnEvent` in OceanBase is classified as an OFFLINE DDL operation, + // and currently, this pipeline connector does not support offline DDL actions. + throw new UnsupportedOperationException("Drop column is not supported currently"); } private void applyRenameColumnEvent(RenameColumnEvent renameColumnEvent) { - // TODO - throw new UnsupportedOperationException("Rename column is not supported currently"); + TableId tableId = renameColumnEvent.tableId(); + Map nameMapping = renameColumnEvent.getNameMapping(); + for (Map.Entry entry : nameMapping.entrySet()) { + catalog.renameColumn( + tableId.getSchemaName(), + tableId.getTableName(), + entry.getKey(), + entry.getValue()); + } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index 26c2007a994..bf8e2aea876 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -49,26 +49,17 @@ public class OceanBaseUtils { /** Convert a source table to {@link OceanBaseTable}. */ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { - List orderedColumns = new ArrayList<>(); - for (String primaryKey : schema.primaryKeys()) { - orderedColumns.add(schema.getColumn(primaryKey).get()); - } - for (Column column : schema.getColumns()) { - if (!schema.primaryKeys().contains(column.getName())) { - orderedColumns.add(column); - } - } - - int primaryKeyCount = schema.primaryKeys().size(); + List columns = schema.getColumns(); + List primaryKeys = schema.primaryKeys(); List oceanBaseColumns = new ArrayList<>(); - for (int i = 0; i < orderedColumns.size(); i++) { - Column column = orderedColumns.get(i); + for (int i = 0; i < columns.size(); i++) { + Column column = columns.get(i); OceanBaseColumn.Builder builder = new OceanBaseColumn.Builder() .setColumnName(column.getName()) .setOrdinalPosition(i) .setColumnComment(column.getComment()); - toOceanBaseDataType(column, i < primaryKeyCount, builder); + toOceanBaseDataType(column, primaryKeys.contains(column.getName()), builder); oceanBaseColumns.add(builder.build()); } @@ -103,7 +94,6 @@ public static void toOceanBaseDataType( public static final String SMALLINT = "SMALLINT"; public static final String INT = "INT"; public static final String BIGINT = "BIGINT"; - public static final String LARGEINT = "BIGINT UNSIGNED"; public static final String FLOAT = "FLOAT"; public static final String DOUBLE = "DOUBLE"; public static final String DECIMAL = "DECIMAL"; @@ -185,24 +175,9 @@ public OceanBaseColumn.Builder visit(DoubleType doubleType) { @Override public OceanBaseColumn.Builder visit(DecimalType decimalType) { - // OceanBase does not support Decimal as primary key, so decimal should be cast to - // VARCHAR. - if (!isPrimaryKeys) { - builder.setDataType(DECIMAL); - builder.setColumnSize(decimalType.getPrecision()); - builder.setDecimalDigits(decimalType.getScale()); - } else { - builder.setDataType(VARCHAR); - // For a DecimalType with precision N, we may need N + 1 or N + 2 characters to - // store it as a - // string (one for negative sign, and one for decimal point) - builder.setColumnSize( - Math.min( - decimalType.getScale() != 0 - ? decimalType.getPrecision() + 2 - : decimalType.getPrecision() + 1, - MAX_VARCHAR_SIZE)); - } + builder.setDataType(DECIMAL); + builder.setColumnSize(decimalType.getPrecision()); + builder.setDecimalDigits(decimalType.getScale()); builder.setNullable(decimalType.isNullable()); return builder; } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index a0153be1845..2943ab4787f 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -29,6 +29,7 @@ import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -143,10 +144,8 @@ public void testSyncWholeDatabase() throws Exception { submitPipelineJob(pipelineJob, mysqlCdcJar, oceanbaseCdcJar, mysqlDriverJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - // waiting for databases were created in oceanbase - Thread.sleep(30_000); - waitForTableCount(MYSQL_TEST_TABLE_NAME, false, 9); + waitingAndAssertTableCount(MYSQL_TEST_TABLE_NAME, false, 9); List originList = queryTable(MYSQL_TEST_TABLE_NAME, false); MatcherAssert.assertThat( originList, @@ -190,7 +189,7 @@ public void testSyncWholeDatabase() throws Exception { throw e; } - waitForTableCount(MYSQL_TEST_TABLE_NAME, false, 10); + waitingAndAssertTableCount(MYSQL_TEST_TABLE_NAME, false, 10); List updateList = queryTable(MYSQL_TEST_TABLE_NAME, false); MatcherAssert.assertThat( updateList, @@ -210,13 +209,20 @@ public void testSyncWholeDatabase() throws Exception { .toArray())); } - private void waitForTableCount(String tableName, boolean isMySQL, int expectedCount) + private void waitingAndAssertTableCount(String tableName, boolean isMySQL, int expectedCount) throws InterruptedException { - while (OceanBaseJdbcUtils.getTableRowsCount( - () -> getConnection(uniqueDatabaseName, isMySQL), tableName) - < expectedCount) { - Thread.sleep(100); + // waiting for databases were created in oceanbase to avoid get connection fail. + Thread.sleep(10_000); + + int tableRowsCount = 0; + for (int i = 0; i < 10; ++i) { + tableRowsCount = + OceanBaseJdbcUtils.getTableRowsCount(() -> getConnection(isMySQL), tableName); + if (tableRowsCount < expectedCount) { + Thread.sleep(100); + } } + Assert.assertEquals(tableRowsCount, expectedCount); } private List queryTable(String tableName, boolean isMySQL) throws SQLException { @@ -226,7 +232,7 @@ private List queryTable(String tableName, boolean isMySQL) throws SQLExc private List queryTable(String tableName, boolean isMySQL, List fields) throws SQLException { List result = new ArrayList<>(); - try (Connection connection = getConnection(tableName, isMySQL); + try (Connection connection = getConnection(isMySQL); Statement statement = connection.createStatement()) { ResultSet rs = statement.executeQuery( @@ -247,7 +253,7 @@ private List queryTable(String tableName, boolean isMySQL, List return result; } - private Connection getConnection(String database, boolean isMySQL) throws SQLException { + private Connection getConnection(boolean isMySQL) throws SQLException { if (isMySQL) { String mysqlJdbcUrl = String.format( From 2e3d19940ed97cd2327be68f23179c441d39e90f Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 24 May 2024 16:31:09 +0800 Subject: [PATCH 06/23] Enhancement: Bump dependency of oceanbase flink connector to 1.2 --- .../flink-cdc-pipeline-connector-oceanbase/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index 926bdd63b15..5b62aeaf953 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -29,7 +29,7 @@ limitations under the License. flink-cdc-pipeline-connector-oceanbase - 1.2-SNAPSHOT + 1.2 From 85a7826b6302baf272e5f8f197feff65aad9cefe Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 27 May 2024 10:00:26 +0800 Subject: [PATCH 07/23] BugFix: add licence header --- .../test/resources/ddl/mysql_2_oceanbase_test.sql | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql index f60ae607a71..42d01aee998 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql @@ -1,3 +1,18 @@ +-- 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. + CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, name VARCHAR(255) NOT NULL DEFAULT 'flink', From 81b537c665affb139276cde5c6b7ef10f6f8ba99 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 27 May 2024 10:49:02 +0800 Subject: [PATCH 08/23] Enhancement: add flink-cdc-pipeline-connector-oceanbase maven module to Github workflow --- .github/workflows/flink_cdc.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/flink_cdc.yml b/.github/workflows/flink_cdc.yml index 85ad4b3b15e..0adca338210 100644 --- a/.github/workflows/flink_cdc.yml +++ b/.github/workflows/flink_cdc.yml @@ -55,7 +55,8 @@ env: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql,\ flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris,\ flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks,\ - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka" + flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka,\ + flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase" MODULES_MYSQL: "\ flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc,\ From 90108542f269e1a03aa72968450b4c8c4bc2de04 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 27 May 2024 13:36:49 +0800 Subject: [PATCH 09/23] Enhancement: refine test case --- .../oceanbase/catalog/OceanBaseMySQLCatalogTest.java | 1 - .../oceanbase/sink/OceanBaseDataSinkFactoryTest.java | 12 +++++++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java index 86e39f820c8..467c8d2718c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -29,7 +29,6 @@ import java.util.List; /** Tests for {@link OceanBaseMySQLCatalogTest}. */ -// @Ignore public class OceanBaseMySQLCatalogTest extends OceanBaseTestBase { private static final ImmutableMap configMap = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java index 2eb7eae33d5..81714dcf12b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkFactoryTest.java @@ -28,6 +28,8 @@ import org.junit.Test; +import java.util.Optional; + import static org.junit.Assert.assertTrue; /** Tests for {@link org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory}. */ @@ -42,9 +44,13 @@ public void testCreateDataSink() { Configuration conf = Configuration.fromMap( ImmutableMap.builder() - .put("url", System.getenv("url")) - .put("username", System.getenv("username")) - .put("password", System.getenv("password")) + .put("url", Optional.ofNullable(System.getenv("url")).orElse("")) + .put( + "username", + Optional.ofNullable(System.getenv("username")).orElse("")) + .put( + "password", + Optional.ofNullable(System.getenv("password")).orElse("")) .build()); DataSink dataSink = sinkFactory.createDataSink( From 9d39771ac17ca1ebb3bfb37839d15e576ab0121d Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 27 May 2024 16:13:56 +0800 Subject: [PATCH 10/23] Enhancement: Add validation to the applyAddColumnEvent method. --- .../oceanbase/sink/OceanBaseMetadataApplier.java | 13 +++++++------ .../cdc/pipeline/tests/OceanBaseE2eITCase.java | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 9666f2f8189..345a658c6be 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -28,6 +28,7 @@ import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalog; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogFactory; @@ -98,13 +99,13 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { List addColumns = new ArrayList<>(); for (AddColumnEvent.ColumnWithPosition columnWithPosition : addColumnEvent.getAddedColumns()) { + Preconditions.checkState( + columnWithPosition.getPosition() == AddColumnEvent.ColumnPosition.LAST, + "The oceanbase pipeline connector currently only supports add the column to the last."); + // we will ignore position information, and always add the column to the last. - // The reason is that the order of columns between source table and OceanBase - // table may be not consistent because of limitations of OceanBase table, so the - // position may be meaningless. For example, primary keys of OceanBase table - // must be at the front, but mysql doest not have this limitation, so the order - // may be different, and also FIRST position is not allowed for OceanBase primary - // key table. + // The reason is that in OceanBase, only adding columns to the last as an online DDL, + // and this pipeline connector currently only supports online DDL. Column column = columnWithPosition.getAddColumn(); OceanBaseColumn.Builder builder = new OceanBaseColumn.Builder() diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 2943ab4787f..073a7073c5e 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -222,7 +222,7 @@ private void waitingAndAssertTableCount(String tableName, boolean isMySQL, int e Thread.sleep(100); } } - Assert.assertEquals(tableRowsCount, expectedCount); + Assert.assertEquals(expectedCount, tableRowsCount); } private List queryTable(String tableName, boolean isMySQL) throws SQLException { From fa9231291e5202bba56214816241c599ba03d8d9 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 27 May 2024 17:12:41 +0800 Subject: [PATCH 11/23] Enhancement: Add a rename column test case to the integration tests. --- .../connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java | 5 ++--- .../connectors/oceanbase/sink/OceanBaseMetadataApplier.java | 3 +-- .../apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java | 1 + 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index 1f09de9de3b..c2e9c5f2a55 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -77,7 +77,7 @@ public boolean databaseExists(String databaseName) throws OceanBaseCatalogExcept * * @param databaseName Name of the database * @param ignoreIfExists Flag to specify behavior when a database with the given name already - * exists: if set to false, throw a StarRocksCatalogException, if set to true, do nothing. + * exists. * @throws OceanBaseCatalogException in case of any runtime exception */ @Override @@ -104,8 +104,7 @@ public void createDatabase(String databaseName, boolean ignoreIfExists) * Creates a table. * * @param table the table definition - * @param ignoreIfExists flag to specify behavior when a table already exists. if set to false, - * it throws a TableAlreadyExistException, if set to true, do nothing. + * @param ignoreIfExists flag to specify behavior when a table already exists. * @throws OceanBaseCatalogException in case of any runtime exception */ @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 345a658c6be..5156dfb51ff 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -58,7 +58,6 @@ public OceanBaseMetadataApplier( @Override public void applySchemaChange(SchemaChangeEvent event) { try { - // send schema change op to doris if (event instanceof CreateTableEvent) { applyCreateTableEvent((CreateTableEvent) event); } else if (event instanceof AddColumnEvent) { @@ -121,7 +120,7 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { } private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { - // TODO The `DropColumnEvent` in OceanBase is classified as an OFFLINE DDL operation, + // TODO The `DropColumnEvent` in OceanBase is classified as an offline DDL operation, // and currently, this pipeline connector does not support offline DDL actions. throw new UnsupportedOperationException("Drop column is not supported currently"); } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 073a7073c5e..50b825ae56b 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -184,6 +184,7 @@ public void testSyncWholeDatabase() throws Exception { "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); stat.execute("UPDATE products SET weight='5.17' WHERE id=111;"); stat.execute("DELETE FROM products WHERE id=111;"); + stat.execute("ALTER TABLE products RENAME COLUMN new_col TO rename_col;"); } catch (SQLException e) { LOG.error("Update table for CDC failed.", e); throw e; From 581faa6fad910e822c563262ff6076a73c7bb310 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Tue, 28 May 2024 13:39:53 +0800 Subject: [PATCH 12/23] Enhancement: remove unnecessary dependencies. --- .../flink-cdc-pipeline-e2e-tests/pom.xml | 14 +++++----- .../pipeline/tests/OceanBaseE2eITCase.java | 28 +++++++++++++++++-- 2 files changed, 32 insertions(+), 10 deletions(-) diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml index 3b855a5e928..a7ee5b79090 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml @@ -96,6 +96,13 @@ limitations under the License. ${project.version} test + + org.apache.flink + flink-cdc-pipeline-connector-oceanbase + ${project.version} + test-jar + test + org.apache.flink flink-connector-test-util @@ -110,13 +117,6 @@ limitations under the License. ${testcontainers.version} test - - org.apache.flink - flink-cdc-pipeline-connector-oceanbase - ${project.version} - test-jar - test - diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 50b825ae56b..53e16570a70 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -23,8 +23,9 @@ import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.connectors.oceanbase.OceanBaseContainer; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.SupplierWithException; -import com.oceanbase.connector.flink.utils.OceanBaseJdbcUtils; import org.apache.commons.text.StringEscapeUtils; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; @@ -217,8 +218,7 @@ private void waitingAndAssertTableCount(String tableName, boolean isMySQL, int e int tableRowsCount = 0; for (int i = 0; i < 10; ++i) { - tableRowsCount = - OceanBaseJdbcUtils.getTableRowsCount(() -> getConnection(isMySQL), tableName); + tableRowsCount = getTableRowsCount(() -> getConnection(isMySQL), tableName); if (tableRowsCount < expectedCount) { Thread.sleep(100); } @@ -268,4 +268,26 @@ private Connection getConnection(boolean isMySQL) throws SQLException { obServer.getUsername(), obServer.getPassword()); } + + private int getTableRowsCount( + SupplierWithException connectionSupplier, String tableName) { + return (int) + query( + connectionSupplier, + "SELECT COUNT(1) FROM " + tableName, + rs -> rs.next() ? rs.getInt(1) : 0); + } + + private Object query( + SupplierWithException connectionSupplier, + String sql, + FunctionWithException resultSetConsumer) { + try (Connection connection = connectionSupplier.get(); + Statement statement = connection.createStatement()) { + ResultSet rs = statement.executeQuery(sql); + return resultSetConsumer.apply(rs); + } catch (SQLException e) { + throw new RuntimeException("Failed to execute sql: " + sql, e); + } + } } From e76f98462c21825a9365c924d9f8a80d587a1002 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Wed, 29 May 2024 17:43:03 +0800 Subject: [PATCH 13/23] Enhancement: add doc for oceanbase-cdc-pipeline-connector --- .../pipeline-connectors/oceanbase.md | 349 ++++++++++++++++++ .../pipeline-connectors/oceanbase.md | 343 +++++++++++++++++ .../oceanbase/catalog/OceanBaseTable.java | 26 +- .../oceanbase/sink/OceanBaseUtils.java | 4 +- 4 files changed, 720 insertions(+), 2 deletions(-) create mode 100644 docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md create mode 100644 docs/content/docs/connectors/pipeline-connectors/oceanbase.md diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md new file mode 100644 index 00000000000..1d74da1324c --- /dev/null +++ b/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md @@ -0,0 +1,349 @@ +--- +title: "OceanBase" +weight: 7 +type: docs +aliases: +- /connectors/pipeline-connectors/oceanbase +--- + + +# OceanBase Connector + +OceanBase Pipeline 连接器可以用作 Pipeline 的 *Data Sink*,将数据写入[OceanBase](https://github.com/oceanbase/oceanbase)。 本文档介绍如何设置 OceanBase Pipeline 连接器。 + +## 连接器的功能 +* 自动建表 +* 表结构变更同步 +* 数据实时同步 + +## 示例 + +从 MySQL 读取数据同步到 OceanBase 的 Pipeline 可以定义如下: + +```yaml +source: + type: mysql + hostname: mysql + port: 3306 + username: mysqluser + password: mysqlpw + tables: mysql_2_oceanbase_test_17l13vc.\.* + server-id: 5400-5404 + server-time-zone: UTC + +sink: + type: oceanbase + url: jdbc:mysql://oceanbase:2881/test + username: root@test + password: + +pipeline: + name: MySQL to OceanBase Pipeline + parallelism: 1 +``` + +## 连接器配置项 + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
参数名是否必需默认值类型描述
typerequired(none)String指定要使用的连接器, 这里需要设置成 'oceanbase'.
urlString数据库的 JDBC url。
usernameString连接用户名。
passwordString连接密码。
schema-nameString连接的 schema 名或 db 名。
table-nameString表名。
driver-class-namecom.mysql.cj.jdbc.DriverString驱动类名,默认为 'com.mysql.cj.jdbc.Driver',如果设置了其他值,需要手动引入对应的依赖。
druid-propertiesStringDruid 连接池属性,多个值用分号分隔。
sync-writefalseBoolean是否开启同步写,设置为 true 时将不使用 buffer 直接写入数据库。
buffer-flush.interval1sDuration缓冲区刷新周期。设置为 '0' 时将关闭定期刷新。
buffer-flush.buffer-size1000Integer缓冲区大小。
max-retries3Integer失败重试次数。
memstore-check.enabledtrueBoolean是否开启内存检查。
memstore-check.threshold0.9Double内存使用的阈值相对最大限制值的比例。
memstore-check.interval30sDuration内存使用检查周期。
partition.enabledfalseBoolean是否启用分区计算功能,按照分区来写数据。仅当 'sync-write' 和 'direct-load.enabled' 都为 false 时生效。
direct-load.enabledfalseBoolean是否开启旁路导入。需要注意旁路导入需要将 sink 的并发度设置为1。
direct-load.hostString旁路导入使用的域名或 IP 地址,开启旁路导入时为必填项。
direct-load.port2882Integer旁路导入使用的 RPC 端口,开启旁路导入时为必填项。
direct-load.parallel8Integer旁路导入任务的并发度。
direct-load.max-error-rows0Long旁路导入任务最大可容忍的错误行数目。
direct-load.dup-actionREPLACESTRING旁路导入任务中主键重复时的处理策略。可以是 'STOP_ON_DUP'(本次导入失败),'REPLACE'(替换)或 'IGNORE'(忽略)。
direct-load.timeout7dDuration旁路导入任务的超时时间。
direct-load.heartbeat-timeout30sDuration旁路导入任务客户端的心跳超时时间。
+
+ +## 使用说明 + +* 暂仅支持OceanBase的MySQL租户 + +* at-least-once语义保证,暂不支持 exactly-once + +* 对于自动建表 + * 没有分区键 + +* 对于表结构变更同步 + * 暂只支持新增列、重命名列 + * 新增列只能添加到最后一列 + +* 对于数据同步,pipeline 连接器使用 [OceanBase Sink 连接器](https://github.com/oceanbase/flink-connector-oceanbase) + 将数据写入 OceanBase,具体可以参考 [Sink 文档](https://github.com/oceanbase/flink-connector-oceanbase/blob/main/docs/sink/flink-connector-oceanbase.md)。 + +## 数据类型映射 +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
CDC typeOceanBase type under MySQL tenantNOTE
TINYINTTINYINT
SMALLINTSMALLINT
INTINT
BIGINTBIGINT
FLOATFLOAT
DOUBLEDOUBLE
DECIMAL(p, s)DECIMAL(p, s)
BOOLEANBOOLEAN
DATEDATE
TIMESTAMPTIMESTAMP
TIMESTAMP_LTZTIMESTAMP
CHAR(n) where n <= 256CHAR(n)
CHAR(n) where n > 256VARCHAR(n)
VARCHAR(n)VARCHAR(n)
+
+ +{{< top >}} diff --git a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md new file mode 100644 index 00000000000..4a247fec7e7 --- /dev/null +++ b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md @@ -0,0 +1,343 @@ +--- +title: "OceanBase" +weight: 6 +type: docs +aliases: +- /connectors/pipeline-connectors/oceanbase +--- + + +# OceanBase Connector + +OceanBase connector can be used as the *Data Sink* of the pipeline, and write data to [OceanBase](https://github.com/oceanbase/oceanbase). This document describes how to set up the OceanBase connector. + +## What can the connector do? +* Create table automatically if not exist +* Schema change synchronization +* Data synchronization + +## Example + +The pipeline for reading data from MySQL and sink to OceanBase can be defined as follows: + +```yaml +source: + type: mysql + hostname: mysql + port: 3306 + username: mysqluser + password: mysqlpw + tables: mysql_2_oceanbase_test_17l13vc.\.* + server-id: 5400-5404 + server-time-zone: UTC + +sink: + type: oceanbase + url: jdbc:mysql://oceanbase:2881/test + username: root@test + password: + +pipeline: + name: MySQL to OceanBase Pipeline + parallelism: 1 +``` + +## Connector Options + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequired by Table APIDefaultTypeDescription
urlYesStringJDBC url.
usernameYesStringThe username.
passwordYesStringThe password.
schema-nameYesStringThe schema name or database name.
table-nameYesStringThe table name.
driver-class-nameNocom.mysql.cj.jdbc.DriverStringThe driver class name, use 'com.mysql.cj.jdbc.Driver' by default. If other value is set, you need to introduce the driver manually.
druid-propertiesNoStringDruid connection pool properties, multiple values are separated by semicolons.
sync-writeNofalseBooleanWhether to write data synchronously, will not use buffer if it's set to 'true'.
buffer-flush.intervalNo1sDurationBuffer flush interval. Set '0' to disable scheduled flushing.
buffer-flush.buffer-sizeNo1000IntegerBuffer size.
max-retriesNo3IntegerMax retry times on failure.
memstore-check.enabledNotrueBooleanWhether enable memstore check.
memstore-check.thresholdNo0.9DoubleMemstore usage threshold ratio relative to the limit value.
memstore-check.intervalNo30sDurationMemstore check interval.
partition.enabledNofalseBooleanWhether to enable partition calculation and flush records by partitions. Only works when 'sync-write' and 'direct-load.enabled' are 'false'.
direct-load.enabledNofalseBooleanWhether to enable direct load. Note that direct load task requires the sink parallelism to be 1.
direct-load.hostNoStringThe hostname or IP address used in direct load task. Required when 'direct-load.enabled' is true.
direct-load.portNo2882IntegerThe rpc port used in direct load task. Required when 'direct-load.enabled' is true.
direct-load.parallelNo8IntegerParallelism of direct load task.
direct-load.max-error-rowsNo0LongMaximum tolerable number of error rows of direct load task.
direct-load.dup-actionNoREPLACEStringAction when there is duplicated record of direct load task. Can be 'STOP_ON_DUP', 'REPLACE' or 'IGNORE'.
direct-load.timeoutNo7dDurationTimeout for direct load task.
direct-load.heartbeat-timeoutNo30sDurationClient heartbeat timeout in direct load task.
+
+ +## Usage Notes + +* Currently only supports MySQL tenants of OceanBase + +* Provides at-least-once semantics, exactly-once is not supported yet + +* For creating table automatically + * there is no partition key + +* For schema change synchronization + * Currently, only adding new columns and renaming columns are supported + * the new column will always be added to the last position + +* For data synchronization, the pipeline connector uses [OceanBase Sink Connector](https://github.com/oceanbase/flink-connector-oceanbase) + to write data to OceanBase. You can see [sink documentation](https://github.com/oceanbase/flink-connector-oceanbase/blob/main/docs/sink/flink-connector-oceanbase.md) + for how it works. + +## Data Type Mapping +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
CDC typeOceanBase type under MySQL tenantNOTE
TINYINTTINYINT
SMALLINTSMALLINT
INTINT
BIGINTBIGINT
FLOATFLOAT
DOUBLEDOUBLE
DECIMAL(p, s)DECIMAL(p, s)
BOOLEANBOOLEAN
DATEDATE
TIMESTAMPTIMESTAMP
TIMESTAMP_LTZTIMESTAMP
CHAR(n) where n <= 256CHAR(n)
CHAR(n) where n > 256VARCHAR(n)
VARCHAR(n)VARCHAR(n)
+
+ +{{< top >}} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java index 9f8080ade6c..2cf5c05391b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseTable.java @@ -52,6 +52,8 @@ public enum TableType { @Nullable private final List tableKeys; + private final List partitionKeys; + /** The table comment. null if there is no comment or it's unknown. */ @Nullable private final String comment; @@ -67,6 +69,7 @@ private OceanBaseTable( TableType tableType, List columns, @Nullable List tableKeys, + @Nullable List partitionKeys, @Nullable String comment, Map properties) { Preconditions.checkNotNull(databaseName); @@ -78,6 +81,7 @@ private OceanBaseTable( this.tableType = tableType; this.columns = columns; this.tableKeys = tableKeys; + this.partitionKeys = partitionKeys; this.comment = comment; this.properties = Preconditions.checkNotNull(properties); } @@ -102,6 +106,10 @@ public Optional> getTableKeys() { return Optional.ofNullable(tableKeys); } + public Optional> getPartitionKeys() { + return Optional.ofNullable(partitionKeys); + } + public Optional getComment() { return Optional.ofNullable(comment); } @@ -139,6 +147,8 @@ public String toString() { + columns + ", tableKeys=" + tableKeys + + ", partitionKeys=" + + partitionKeys + ", comment='" + comment + '\'' @@ -161,6 +171,7 @@ public boolean equals(Object o) { && tableType == that.tableType && Objects.equals(columns, that.columns) && Objects.equals(tableKeys, that.tableKeys) + && Objects.equals(partitionKeys, that.partitionKeys) && Objects.equals(comment, that.comment) && Objects.equals(properties, that.properties); } @@ -173,6 +184,7 @@ public static class Builder { private TableType tableType; private List columns = new ArrayList<>(); private List tableKeys; + private List partitionKeys; private String comment; private Map properties = new HashMap<>(); @@ -201,6 +213,11 @@ public Builder setTableKeys(List tableKeys) { return this; } + public Builder setPartitionKeys(List partitionKeys) { + this.partitionKeys = partitionKeys; + return this; + } + public Builder setComment(String comment) { this.comment = comment; return this; @@ -213,7 +230,14 @@ public Builder setTableProperties(Map properties) { public OceanBaseTable build() { return new OceanBaseTable( - databaseName, tableName, tableType, columns, tableKeys, comment, properties); + databaseName, + tableName, + tableType, + columns, + tableKeys, + partitionKeys, + comment, + properties); } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index bf8e2aea876..2d943537398 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -73,6 +73,7 @@ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { : OceanBaseTable.TableType.DUPLICATE_KEY) .setColumns(oceanBaseColumns) .setTableKeys(schema.primaryKeys()) + .setPartitionKeys(schema.partitionKeys()) .setComment(schema.comment()); return tableBuilder.build(); } @@ -102,6 +103,7 @@ public static void toOceanBaseDataType( public static final String STRING = "STRING"; public static final String DATE = "DATE"; public static final String DATETIME = "DATETIME"; + public static final String TIMESTAMP = "TIMESTAMP"; public static final String JSON = "JSON"; public static final String TEXT = "TEXT"; public static final String BLOB = "BLOB"; @@ -220,7 +222,7 @@ public OceanBaseColumn.Builder visit(DateType dateType) { @Override public OceanBaseColumn.Builder visit(TimestampType timestampType) { - builder.setDataType(DATETIME); + builder.setDataType(TIMESTAMP); builder.setNullable(timestampType.isNullable()); return builder; } From a1bd106a13db17c889289bf77af5963e34dc168e Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Thu, 30 May 2024 18:22:54 +0800 Subject: [PATCH 14/23] Enhancement: Improve documentation and integration tests. --- .../pipeline-connectors/overview.md | 1 + .../pipeline-connectors/oceanbase.md | 2 +- .../pipeline-connectors/overview.md | 1 + .../catalog/OceanBaseMySQLCatalog.java | 4 ++ .../oceanbase/sink/OceanBaseDataSink.java | 3 +- .../oceanbase/sink/OceanBaseUtils.java | 2 + .../pipeline/tests/OceanBaseE2eITCase.java | 40 ++++++++++++++----- .../resources/ddl/mysql_2_oceanbase_test.sql | 17 +++++++- 8 files changed, 57 insertions(+), 13 deletions(-) diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/overview.md b/docs/content.zh/docs/connectors/pipeline-connectors/overview.md index 80c8251b416..b9a884982a8 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/overview.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/overview.md @@ -37,6 +37,7 @@ Flink CDC 提供了可用于 YAML 作业的 Pipeline Source 和 Sink 连接器 | [MySQL]({{< ref "docs/connectors/pipeline-connectors/mysql" >}}) | Source |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | | [Paimon]({{< ref "docs/connectors/pipeline-connectors/paimon" >}}) | Sink |
  • [Paimon](https://paimon.apache.org/): 0.6, 0.7, 0.8 | | [StarRocks]({{< ref "docs/connectors/pipeline-connectors/starrocks" >}}) | Sink |
  • [StarRocks](https://www.starrocks.io/): 2.x, 3.x | +| [OceanBase]({{< ref "docs/connectors/pipeline-connectors/oceanbase" >}}) | Sink |
  • [OceanBase](https://www.oceanbase.com/): 3.x, 4.x | ## Develop Your Own Connector diff --git a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md index 4a247fec7e7..a209a0e3156 100644 --- a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md +++ b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md @@ -1,6 +1,6 @@ --- title: "OceanBase" -weight: 6 +weight: 7 type: docs aliases: - /connectors/pipeline-connectors/oceanbase diff --git a/docs/content/docs/connectors/pipeline-connectors/overview.md b/docs/content/docs/connectors/pipeline-connectors/overview.md index 2d536a68f6e..e1be68803e9 100644 --- a/docs/content/docs/connectors/pipeline-connectors/overview.md +++ b/docs/content/docs/connectors/pipeline-connectors/overview.md @@ -40,6 +40,7 @@ definition. | [MySQL]({{< ref "docs/connectors/pipeline-connectors/mysql" >}}) | Source |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | | [Paimon]({{< ref "docs/connectors/pipeline-connectors/paimon" >}}) | Sink |
  • [Paimon](https://paimon.apache.org/): 0.6, 0.7, 0.8 | | [StarRocks]({{< ref "docs/connectors/pipeline-connectors/starrocks" >}}) | Sink |
  • [StarRocks](https://www.starrocks.io/): 2.x, 3.x | +| [OceanBase]({{< ref "docs/connectors/pipeline-connectors/oceanbase" >}}) | Sink |
  • [OceanBase](https://www.oceanbase.com/): 3.x, 4.x | ## Develop Your Own Connector diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index c2e9c5f2a55..3bdda647faf 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -281,6 +281,10 @@ protected String getFullColumnType( Preconditions.checkArgument( columnSize.isPresent(), type + " type must have column size"); return String.format("%s(%d)", dataType, columnSize.get()); + case "TIMESTAMP": + return columnSize + .map(size -> String.format("%s(%d)", dataType, size)) + .orElse(dataType); default: return dataType; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java index 3605bd75c43..1e6c4d01909 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -22,6 +22,7 @@ import org.apache.flink.cdc.common.sink.EventSinkProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; @@ -68,7 +69,7 @@ public MetadataApplier getMetadataApplier() { try { return new OceanBaseMetadataApplier(connectorOptions, config); } catch (Exception e) { - throw new RuntimeException(e.getMessage(), e); + throw new OceanBaseCatalogException("Fail to create oceanbase metadata applier.", e); } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index 2d943537398..e36c9913ad8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -224,6 +224,8 @@ public OceanBaseColumn.Builder visit(DateType dateType) { public OceanBaseColumn.Builder visit(TimestampType timestampType) { builder.setDataType(TIMESTAMP); builder.setNullable(timestampType.isNullable()); + builder.setColumnSize( + Math.min(timestampType.getPrecision(), TimestampType.DEFAULT_PRECISION)); return builder; } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 53e16570a70..1d588b54d28 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -33,7 +33,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -90,8 +89,8 @@ public class OceanBaseE2eITCase extends PipelineTestEnvironment { private final String uniqueDatabaseName = mysqlInventoryDatabase.getDatabaseName(); - @Rule - public final OceanBaseContainer obServer = + @ClassRule + public static final OceanBaseContainer OB_SERVER = new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) .withNetwork(NETWORK) .withNetworkAliases("oceanbase") @@ -136,9 +135,9 @@ public void testSyncWholeDatabase() throws Exception { MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, uniqueDatabaseName, - obServer.getJdbcUrlInContainer("test", "oceanbase"), - obServer.getUsername(), - obServer.getPassword()); + OB_SERVER.getJdbcUrlInContainer("test", "oceanbase"), + OB_SERVER.getUsername(), + OB_SERVER.getPassword()); Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path oceanbaseCdcJar = TestUtils.getResource("oceanbase-cdc-pipeline-connector.jar"); Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); @@ -163,6 +162,17 @@ public void testSyncWholeDatabase() throws Exception { "109,spare tire,24 inch spare tire,22.2,null,null,null") .map(StringEscapeUtils::unescapeJava) .toArray())); + // validate table of customers + List customerList = queryTable("customers", false); + MatcherAssert.assertThat( + customerList, + Matchers.containsInAnyOrder( + Stream.of( + "101,user_1,Shanghai,123567891234,2023-12-12 11:00:11.0", + "102,user_2,Shanghai,123567891234,2023-12-12 11:00:11.0", + "103,user_3,Shanghai,123567891234,2023-12-12 11:00:11.0", + "104,user_4,Shanghai,123567891234,2023-12-12 11:00:11.0") + .toArray())); // generate binlogs String mysqlJdbcUrl = @@ -209,6 +219,8 @@ public void testSyncWholeDatabase() throws Exception { "110,jacket,new water resistent white wind breaker,0.5,null,null,null,1") .map(StringEscapeUtils::unescapeJava) .toArray())); + + dropDatabase(getConnection(false), uniqueDatabaseName); } private void waitingAndAssertTableCount(String tableName, boolean isMySQL, int expectedCount) @@ -260,13 +272,12 @@ private Connection getConnection(boolean isMySQL) throws SQLException { String.format( "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), uniqueDatabaseName); - System.out.println(mysqlJdbcUrl); return DriverManager.getConnection(mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); } return DriverManager.getConnection( - obServer.getJdbcUrl(uniqueDatabaseName), - obServer.getUsername(), - obServer.getPassword()); + OB_SERVER.getJdbcUrl(uniqueDatabaseName), + OB_SERVER.getUsername(), + OB_SERVER.getPassword()); } private int getTableRowsCount( @@ -290,4 +301,13 @@ private Object query( throw new RuntimeException("Failed to execute sql: " + sql, e); } } + + private void dropDatabase(Connection connection, String database) { + try (Connection conn = connection; + Statement statement = conn.createStatement()) { + statement.execute(String.format("DROP DATABASE %s", database)); + } catch (SQLException e) { + throw new RuntimeException("Failed to drop database", e); + } + } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql index 42d01aee998..5d43ff0fb4f 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/mysql_2_oceanbase_test.sql @@ -33,4 +33,19 @@ VALUES (default,"scooter","Small 2-wheel scooter",3.14, 'red', '{"key1": "value1 (default,"hammer","16oz carpenter's hammer",1.0, null, null, null), (default,"rocks","box of assorted rocks",5.3, null, null, null), (default,"jacket","water resistent black wind breaker",0.1, null, null, null), - (default,"spare tire","24 inch spare tire",22.2, null, null, null); \ No newline at end of file + (default,"spare tire","24 inch spare tire",22.2, null, null, null); + + +CREATE TABLE customers ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512), + create_date DATETIME +); + +INSERT INTO customers +VALUES (101,"user_1","Shanghai","123567891234", "2023-12-12 11:00:11"), + (102,"user_2","Shanghai","123567891234", "2023-12-12 11:00:11"), + (103,"user_3","Shanghai","123567891234", "2023-12-12 11:00:11"), + (104,"user_4","Shanghai","123567891234", "2023-12-12 11:00:11"); \ No newline at end of file From 75298b5b5684a822d14e96afbdc0268a0cfb35e4 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Thu, 13 Jun 2024 15:40:30 +0800 Subject: [PATCH 15/23] Enhancement: Refine code according to CR. --- .../oceanbase/sink/OceanBaseDataSinkOptions.java | 9 +++++++++ .../sink/OceanBaseEventSerializationSchema.java | 3 +++ 2 files changed, 12 insertions(+) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java index c6183ed59a0..dfdcaf22ce2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; +import org.apache.flink.cdc.common.annotation.Experimental; import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.ConfigOptions; @@ -87,48 +88,56 @@ public class OceanBaseDataSinkOptions { .withDescription( "Whether to enable partition calculation and flush records by partitions. Default value is 'false'."); + @Experimental public static final ConfigOption DIRECT_LOAD_ENABLED = ConfigOptions.key("direct-load.enabled") .booleanType() .defaultValue(false) .withDescription("Whether to enable direct load."); + @Experimental public static final ConfigOption DIRECT_LOAD_HOST = ConfigOptions.key("direct-load.host") .stringType() .noDefaultValue() .withDescription("Hostname used in direct load."); + @Experimental public static final ConfigOption DIRECT_LOAD_PORT = ConfigOptions.key("direct-load.port") .intType() .defaultValue(2882) .withDescription("Rpc port number used in direct load."); + @Experimental public static final ConfigOption DIRECT_LOAD_PARALLEL = ConfigOptions.key("direct-load.parallel") .intType() .defaultValue(8) .withDescription("Parallelism of direct load."); + @Experimental public static final ConfigOption DIRECT_LOAD_MAX_ERROR_ROWS = ConfigOptions.key("direct-load.max-error-rows") .longType() .defaultValue(0L) .withDescription("Maximum tolerable number of error rows."); + @Experimental public static final ConfigOption DIRECT_LOAD_DUP_ACTION = ConfigOptions.key("direct-load.dup-action") .enumType(ObLoadDupActionType.class) .defaultValue(ObLoadDupActionType.REPLACE) .withDescription("Action when there is duplicated record in direct load."); + @Experimental public static final ConfigOption DIRECT_LOAD_TIMEOUT = ConfigOptions.key("direct-load.timeout") .durationType() .defaultValue(Duration.ofDays(7)) .withDescription("Timeout for direct load task."); + @Experimental public static final ConfigOption DIRECT_LOAD_HEARTBEAT_TIMEOUT = ConfigOptions.key("direct-load.heartbeat-timeout") .durationType() diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index 94c40664f6b..dd4ae3b1348 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -40,6 +40,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; /** A serializer for Event to Record. */ public class OceanBaseEventSerializationSchema implements RecordSerializationSchema { @@ -100,6 +101,8 @@ private Record applyDataChangeEvent(DataChangeEvent event) { private DataChangeRecord buildDataChangeRecord( TableId tableId, Schema schema, Object[] values, boolean isDelete) { + Preconditions.checkState( + Objects.nonNull(tableId.getSchemaName()), "schema name cannot be null or empty."); com.oceanbase.connector.flink.table.TableId oceanBaseTableId = new com.oceanbase.connector.flink.table.TableId( tableId.getSchemaName(), tableId.getTableName()); From 285d9c5a161758af5524f5f2314553301ea7ef4d Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Thu, 13 Jun 2024 17:41:16 +0800 Subject: [PATCH 16/23] Enhancement: Remove unnecessary container startup in unit tests. --- .../pom.xml | 12 ++++++++++++ .../oceanbase/catalog/OceanBaseCatalog.java | 16 +++++++++++++++- .../oceanbase/sink/OceanBaseMetadataApplier.java | 1 + .../catalog/OceanBaseMySQLCatalogTest.java | 10 ++++------ 4 files changed, 32 insertions(+), 7 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index 5b62aeaf953..f0ae2b55429 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -68,6 +68,18 @@ limitations under the License. + + org.apache.maven.plugins + maven-jar-plugin + + + test-jar + + test-jar + + + + org.apache.maven.plugins maven-shade-plugin diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java index ccdc69e39b6..11aacf09f7a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java @@ -20,6 +20,8 @@ import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; import org.apache.commons.compress.utils.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Serializable; import java.sql.Connection; @@ -33,11 +35,19 @@ /** A {@link OceanBaseCatalog} for OceanBase connector that supports schema evolution. */ public abstract class OceanBaseCatalog implements Serializable { private static final long serialVersionUID = 1L; - private final OceanBaseConnectionProvider connectionProvider; + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseCatalog.class); + + private OceanBaseConnectionProvider connectionProvider; + private final OceanBaseConnectorOptions connectorOptions; public OceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) { assert Objects.nonNull(connectorOptions); + this.connectorOptions = connectorOptions; + } + + public void open() { this.connectionProvider = new OceanBaseConnectionProvider(connectorOptions); + LOG.info("Open OceanBase catalog"); } protected List executeSingleColumnStatement(String sql) throws SQLException { @@ -74,4 +84,8 @@ public abstract void alterAddColumns( public abstract void renameColumn( String schemaName, String tableName, String oldColumnName, String newColumnName); + + public void close() { + LOG.info("Close OceanBase catalog"); + } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 5156dfb51ff..19dfd2af9d4 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -53,6 +53,7 @@ public class OceanBaseMetadataApplier implements MetadataApplier { public OceanBaseMetadataApplier( OceanBaseConnectorOptions connectorOptions, Configuration config) throws Exception { this.catalog = OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); + catalog.open(); } @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java index 467c8d2718c..bae409d29d5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalogTest.java @@ -17,8 +17,6 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; - import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; @@ -29,13 +27,13 @@ import java.util.List; /** Tests for {@link OceanBaseMySQLCatalogTest}. */ -public class OceanBaseMySQLCatalogTest extends OceanBaseTestBase { +public class OceanBaseMySQLCatalogTest { private static final ImmutableMap configMap = ImmutableMap.builder() - .put("url", OB_SERVER.getJdbcUrl()) - .put("username", OB_SERVER.getUsername()) - .put("password", OB_SERVER.getPassword()) + .put("url", "localhost") + .put("username", "test") + .put("password", "test") .build(); @Test From 408f1a302946bf4b4e3b430d27382e268c872cc3 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 14 Jun 2024 11:03:39 +0800 Subject: [PATCH 17/23] doc: Fix the incorrect description of driver in the documentation. --- .../content.zh/docs/connectors/pipeline-connectors/oceanbase.md | 2 +- docs/content/docs/connectors/pipeline-connectors/oceanbase.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md index 1d74da1324c..17f4f91f62f 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/oceanbase.md @@ -120,7 +120,7 @@ pipeline: 否 com.mysql.cj.jdbc.Driver String - 驱动类名,默认为 'com.mysql.cj.jdbc.Driver',如果设置了其他值,需要手动引入对应的依赖。 + 驱动类名,默认为 'com.mysql.cj.jdbc.Driver'。同时该connector并不包含对应驱动,需手动引入。 druid-properties diff --git a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md index a209a0e3156..a41d93d95b8 100644 --- a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md +++ b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md @@ -113,7 +113,7 @@ pipeline: No com.mysql.cj.jdbc.Driver String - The driver class name, use 'com.mysql.cj.jdbc.Driver' by default. If other value is set, you need to introduce the driver manually. + The driver class name, use 'com.mysql.cj.jdbc.Driver' by default. And the connector does not contain the corresponding driver and needs to be introduced manually. druid-properties From 78ab2a8f57dfaa8328f0081701afd0b11fb1e309 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 17 Jun 2024 10:32:55 +0800 Subject: [PATCH 18/23] Enhancement: Fix code according to CR. --- .../catalog/OceanBaseCatalogFactory.java | 3 ++- .../factory/OceanBaseDataSinkFactory.java | 23 ++++++++++++------- .../oceanbase/sink/OceanBaseDataSink.java | 14 ++--------- .../OceanBaseEventSerializationSchema.java | 4 ++-- .../sink/OceanBaseMetadataApplier.java | 12 ++++++---- 5 files changed, 28 insertions(+), 28 deletions(-) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java index 81f49c093b6..4b6e94fcb02 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalogFactory.java @@ -36,7 +36,8 @@ public static OceanBaseCatalog createOceanBaseCatalog( } else if (dialect instanceof OceanBaseOracleDialect) { return new OceanBaseOracleCatalog(connectorOptions); } else { - throw new OceanBaseCatalogException("This tenant is not supported currently"); + throw new OceanBaseCatalogException( + "Fail to create OceanBaseCatalog: unknown tenant."); } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index bbd6c248ed0..bbf39806c19 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -24,8 +24,10 @@ import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSink; import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseDataSinkOptions; +import org.apache.flink.cdc.connectors.oceanbase.utils.OptionUtils; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.apache.commons.lang3.StringUtils; import java.time.ZoneId; import java.util.HashSet; @@ -39,30 +41,35 @@ @Internal public class OceanBaseDataSinkFactory implements DataSinkFactory { + private static final String IDENTIFIER = "oceanbase"; + @Override public DataSink createDataSink(Context context) { Configuration config = context.getFactoryConfiguration(); - OceanBaseConnectorOptions connectorOptions = - new OceanBaseConnectorOptions(buildOceanBaseOptions(config)); - String zoneStr = context.getFactoryConfiguration().get(PIPELINE_LOCAL_TIME_ZONE); + Map configMap = buildOceanBaseOptions(config); + OptionUtils.printOptions(IDENTIFIER, configMap); + + OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(configMap); + String zoneStr = context.getPipelineConfiguration().get(PIPELINE_LOCAL_TIME_ZONE); ZoneId zoneId = PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(zoneStr) ? ZoneId.systemDefault() : ZoneId.of(zoneStr); - return new OceanBaseDataSink(connectorOptions, config, zoneId); + return new OceanBaseDataSink(connectorOptions, zoneId); } public Map buildOceanBaseOptions(Configuration config) { Optional optional = config.getOptional(OceanBaseDataSinkOptions.PASSWORD); + // Avoid NullPointerException when PASSWORD is empty. config.remove(OceanBaseDataSinkOptions.PASSWORD); - Map map = config.toMap(); - map.put(OceanBaseDataSinkOptions.PASSWORD.key(), optional.orElse("")); - return map; + Map configMap = config.toMap(); + configMap.put(OceanBaseDataSinkOptions.PASSWORD.key(), optional.orElse(StringUtils.EMPTY)); + return configMap; } @Override public String identifier() { - return "oceanbase"; + return IDENTIFIER; } @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java index 1e6c4d01909..b9ec6d3044d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSink.java @@ -17,12 +17,10 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; -import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; import org.apache.flink.cdc.common.sink.MetadataApplier; -import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; @@ -38,14 +36,10 @@ public class OceanBaseDataSink implements DataSink, Serializable { private final OceanBaseConnectorOptions connectorOptions; - private final Configuration config; - private final ZoneId zoneId; - public OceanBaseDataSink( - OceanBaseConnectorOptions options, Configuration config, ZoneId zoneId) { + public OceanBaseDataSink(OceanBaseConnectorOptions options, ZoneId zoneId) { this.connectorOptions = options; - this.config = config; this.zoneId = zoneId; } @@ -66,10 +60,6 @@ public EventSinkProvider getEventSinkProvider() { @Override public MetadataApplier getMetadataApplier() { - try { - return new OceanBaseMetadataApplier(connectorOptions, config); - } catch (Exception e) { - throw new OceanBaseCatalogException("Fail to create oceanbase metadata applier.", e); - } + return new OceanBaseMetadataApplier(connectorOptions); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index dd4ae3b1348..b94bb20e6dd 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -94,7 +94,7 @@ private Record applyDataChangeEvent(DataChangeEvent event) { isDelete = true; break; default: - throw new UnsupportedOperationException("Unsupport Operation " + op); + throw new UnsupportedOperationException("Unsupported Operation " + op); } return buildDataChangeRecord(tableId, schema, values, isDelete); } @@ -102,7 +102,7 @@ private Record applyDataChangeEvent(DataChangeEvent event) { private DataChangeRecord buildDataChangeRecord( TableId tableId, Schema schema, Object[] values, boolean isDelete) { Preconditions.checkState( - Objects.nonNull(tableId.getSchemaName()), "schema name cannot be null or empty."); + Objects.nonNull(tableId.getSchemaName()), "Schema name cannot be null or empty."); com.oceanbase.connector.flink.table.TableId oceanBaseTableId = new com.oceanbase.connector.flink.table.TableId( tableId.getSchemaName(), tableId.getTableName()); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 19dfd2af9d4..8ada243115e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; -import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.AddColumnEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -50,10 +49,13 @@ public class OceanBaseMetadataApplier implements MetadataApplier { private final OceanBaseCatalog catalog; - public OceanBaseMetadataApplier( - OceanBaseConnectorOptions connectorOptions, Configuration config) throws Exception { - this.catalog = OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); - catalog.open(); + public OceanBaseMetadataApplier(OceanBaseConnectorOptions connectorOptions) { + try { + this.catalog = OceanBaseCatalogFactory.createOceanBaseCatalog(connectorOptions); + catalog.open(); + } catch (Exception e) { + throw new OceanBaseCatalogException("Fail to init OceanBaseMetadataApplier.", e); + } } @Override From 02889e830a91e24bae687d48c8dd6a22f0fa6d34 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Tue, 18 Jun 2024 14:51:10 +0800 Subject: [PATCH 19/23] BugFix: git commit file: OptionUtils. --- .../oceanbase/utils/OptionUtils.java | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java new file mode 100644 index 00000000000..a71dd26a627 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java @@ -0,0 +1,42 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.utils; + +import org.apache.flink.configuration.ConfigurationUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** A utility class to print configuration of connectors. */ +public class OptionUtils { + + private static final Logger LOG = LoggerFactory.getLogger(OptionUtils.class); + + /** Utility class can not be instantiated. */ + private OptionUtils() {} + + public static void printOptions(String identifier, Map config) { + Map hideMap = ConfigurationUtils.hideSensitiveValues(config); + LOG.info("Print {} connector configuration:", identifier); + for (String key : hideMap.keySet()) { + LOG.info("{} = {}", key, hideMap.get(key)); + } + } +} From aa3ecd4ff715ca7d017218fa92d08c67f2c10056 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Tue, 20 Aug 2024 09:54:03 +0800 Subject: [PATCH 20/23] =?UTF-8?q?Enhancement=EF=BC=9ARefactored=20the=20oc?= =?UTF-8?q?eanbase=20container=20creation=20logic.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../factory/OceanBaseDataSinkFactory.java | 13 +- .../oceanbase/OceanBaseContainer.java | 142 ------------------ .../oceanbase/OceanBaseTestBase.java | 49 ------ .../flink-cdc-pipeline-e2e-tests/pom.xml | 2 +- .../pipeline/tests/OceanBaseE2eITCase.java | 14 +- 5 files changed, 10 insertions(+), 210 deletions(-) delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java delete mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index bbf39806c19..ca82835c54e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -27,12 +27,10 @@ import org.apache.flink.cdc.connectors.oceanbase.utils.OptionUtils; import com.oceanbase.connector.flink.OceanBaseConnectorOptions; -import org.apache.commons.lang3.StringUtils; import java.time.ZoneId; import java.util.HashSet; import java.util.Map; -import java.util.Optional; import java.util.Set; import static org.apache.flink.cdc.common.pipeline.PipelineOptions.PIPELINE_LOCAL_TIME_ZONE; @@ -46,7 +44,7 @@ public class OceanBaseDataSinkFactory implements DataSinkFactory { @Override public DataSink createDataSink(Context context) { Configuration config = context.getFactoryConfiguration(); - Map configMap = buildOceanBaseOptions(config); + Map configMap = config.toMap(); OptionUtils.printOptions(IDENTIFIER, configMap); OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(configMap); @@ -58,15 +56,6 @@ public DataSink createDataSink(Context context) { return new OceanBaseDataSink(connectorOptions, zoneId); } - public Map buildOceanBaseOptions(Configuration config) { - Optional optional = config.getOptional(OceanBaseDataSinkOptions.PASSWORD); - // Avoid NullPointerException when PASSWORD is empty. - config.remove(OceanBaseDataSinkOptions.PASSWORD); - Map configMap = config.toMap(); - configMap.put(OceanBaseDataSinkOptions.PASSWORD.key(), optional.orElse(StringUtils.EMPTY)); - return configMap; - } - @Override public String identifier() { return IDENTIFIER; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java deleted file mode 100644 index 548e36e6bac..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseContainer.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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. - */ - -package org.apache.flink.cdc.connectors.oceanbase; - -import org.testcontainers.containers.JdbcDatabaseContainer; -import org.testcontainers.containers.wait.strategy.Wait; -import org.testcontainers.utility.DockerImageName; - -import java.time.Duration; - -/** A {@link org.testcontainers.containers.Container} for "OceanBase". */ -public class OceanBaseContainer extends JdbcDatabaseContainer { - - public static final String DOCKER_IMAGE_NAME = "oceanbase/oceanbase-ce"; - - private static final DockerImageName DEFAULT_IMAGE_NAME = - DockerImageName.parse(DOCKER_IMAGE_NAME); - - private static final Integer SQL_PORT = 2881; - private static final Integer RPC_PORT = 2882; - - private static final String DEFAULT_USERNAME = "root"; - private static final String DEFAULT_PASSWORD = ""; - private static final String DEFAULT_TENANT_NAME = "test"; - private static final String DEFAULT_DATABASE_NAME = "test"; - - private String sysPassword = DEFAULT_PASSWORD; - - public OceanBaseContainer(String dockerImageName) { - this(DockerImageName.parse(dockerImageName)); - } - - public OceanBaseContainer(DockerImageName dockerImageName) { - super(dockerImageName); - dockerImageName.assertCompatibleWith(DEFAULT_IMAGE_NAME); - - this.waitStrategy = - Wait.forLogMessage(".*boot success!.*", 1) - .withStartupTimeout(Duration.ofMinutes(5)); - - addExposedPorts(SQL_PORT, RPC_PORT); - } - - @Override - public String getDriverClassName() { - return "com.oceanbase.jdbc.Driver"; - } - - public Integer getSqlPort() { - return getActualPort(SQL_PORT); - } - - public Integer getActualPort(int port) { - return "host".equals(getNetworkMode()) ? port : getMappedPort(port); - } - - @Override - public String getJdbcUrl() { - return getJdbcUrl(DEFAULT_DATABASE_NAME); - } - - public String getJdbcUrl(String databaseName) { - String additionalUrlParams = constructUrlParameters("?", "&"); - return "jdbc:mysql://" - + getHost() - + ":" - + getSqlPort() - + "/" - + databaseName - + additionalUrlParams; - } - - public String getJdbcUrlInContainer(String databaseName, String networkAliases) { - String additionalUrlParams = constructUrlParameters("?", "&"); - return "jdbc:mysql://" - + networkAliases - + ":" - + SQL_PORT - + "/" - + databaseName - + additionalUrlParams; - } - - public OceanBaseContainer withSysPassword(String sysPassword) { - this.sysPassword = sysPassword; - return this; - } - - public String getSysUsername() { - return DEFAULT_USERNAME; - } - - public String getSysPassword() { - return sysPassword; - } - - @Override - public String getDatabaseName() { - return DEFAULT_DATABASE_NAME; - } - - @Override - public String getUsername() { - return DEFAULT_USERNAME + "@" + DEFAULT_TENANT_NAME; - } - - @Override - public String getPassword() { - return DEFAULT_PASSWORD; - } - - @Override - protected String getTestQueryString() { - return "SELECT 1"; - } - - @Override - protected void waitUntilContainerStarted() { - getWaitStrategy().waitUntilReady(this); - } - - @Override - protected void configure() { - withEnv("MODE", "slim"); - withEnv("OB_ROOT_PASSWORD", sysPassword); - } -} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java deleted file mode 100644 index 73109d74214..00000000000 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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. - */ - -package org.apache.flink.cdc.connectors.oceanbase; - -import org.junit.ClassRule; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.MountableFile; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; - -/** Basic class for testing OceanBase. */ -public class OceanBaseTestBase { - private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestBase.class); - - public static final String IMAGE_TAG = "4.2.1_bp2"; - - @ClassRule - public static final OceanBaseContainer OB_SERVER = - new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) - .withSysPassword("123456") - .withCopyFileToContainer( - MountableFile.forClasspathResource("sql/init.sql"), - "/root/boot/init.d/init.sql") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - protected Connection getConnection() throws SQLException { - return DriverManager.getConnection( - OB_SERVER.getJdbcUrl(), OB_SERVER.getUsername(), OB_SERVER.getPassword()); - } -} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml index 25e2052442c..50a3c073dae 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/pom.xml @@ -102,7 +102,7 @@ limitations under the License. org.apache.flink - flink-cdc-pipeline-connector-oceanbase + flink-connector-oceanbase-cdc ${project.version} test-jar test diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 1d588b54d28..dbeb50dfee4 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -21,7 +21,8 @@ import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; -import org.apache.flink.cdc.connectors.oceanbase.OceanBaseContainer; +import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils; +import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; import org.apache.flink.util.function.FunctionWithException; import org.apache.flink.util.function.SupplierWithException; @@ -53,8 +54,6 @@ import java.util.List; import java.util.stream.Stream; -import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase.IMAGE_TAG; - /** OceanBase flink cdc pipeline connector sink integrate test. */ @RunWith(Parameterized.class) public class OceanBaseE2eITCase extends PipelineTestEnvironment { @@ -91,10 +90,9 @@ public class OceanBaseE2eITCase extends PipelineTestEnvironment { @ClassRule public static final OceanBaseContainer OB_SERVER = - new OceanBaseContainer(OceanBaseContainer.DOCKER_IMAGE_NAME + ":" + IMAGE_TAG) + OceanBaseTestUtils.createOceanBaseContainerForJdbc() .withNetwork(NETWORK) .withNetworkAliases("oceanbase") - .withSysPassword("123456") .withLogConsumer(new Slf4jLogConsumer(LOG)); @Before @@ -135,7 +133,7 @@ public void testSyncWholeDatabase() throws Exception { MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, uniqueDatabaseName, - OB_SERVER.getJdbcUrlInContainer("test", "oceanbase"), + getJdbcUrlInContainer("test", "oceanbase"), OB_SERVER.getUsername(), OB_SERVER.getPassword()); Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); @@ -310,4 +308,8 @@ private void dropDatabase(Connection connection, String database) { throw new RuntimeException("Failed to drop database", e); } } + + public String getJdbcUrlInContainer(String databaseName, String networkAliases) { + return "jdbc:mysql://" + networkAliases + ":" + 2881 + "/" + databaseName; + } } From 0551f4fbdb8d9d460d8f885063b94eb522da0490 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Fri, 8 Nov 2024 17:22:01 +0800 Subject: [PATCH 21/23] =?UTF-8?q?Enhancement=EF=BC=9AResolved=20conflicts.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .github/workflows/flink_cdc_base.yml | 3 ++- .../flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/flink_cdc_base.yml b/.github/workflows/flink_cdc_base.yml index 74260ef1404..f2c60bd4c2b 100644 --- a/.github/workflows/flink_cdc_base.yml +++ b/.github/workflows/flink_cdc_base.yml @@ -50,7 +50,8 @@ env: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks,\ flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka,\ flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon,\ - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch" + flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch,\ + flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase" MODULES_MYSQL: "\ flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc,\ diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java index 50885089e63..a8eec4f859f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java @@ -45,7 +45,7 @@ public static OceanBaseContainer createOceanBaseContainerForCDC() { } public static OceanBaseContainer createOceanBaseContainerForJdbc() { - return createOceanBaseContainer(LATEST_VERSION, "slim") + return createOceanBaseContainer(LATEST_VERSION, "mini") .withStartupTimeout(Duration.ofMinutes(2)); } From 0654604cfcbe4a9b84d09454bad568d27c019a9d Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Mon, 18 Nov 2024 13:41:06 +0800 Subject: [PATCH 22/23] =?UTF-8?q?Enhancement=EF=BC=9AFix=20code.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../pipeline-connectors/oceanbase.md | 56 ---- .../pom.xml | 21 ++ .../oceanbase/catalog/OceanBaseCatalog.java | 17 +- .../oceanbase/catalog/OceanBaseColumn.java | 31 +-- .../catalog/OceanBaseMySQLCatalog.java | 187 ++++++++++++- .../catalog/OceanBaseOracleCatalog.java | 34 +++ .../factory/OceanBaseDataSinkFactory.java | 8 - .../sink/OceanBaseDataSinkOptions.java | 59 ---- .../sink/OceanBaseMetadataApplier.java | 91 +++++-- .../oceanbase/sink/OceanBaseUtils.java | 33 ++- .../connectors/oceanbase/utils/ListUtils.java | 33 +++ ...OceanBaseEventSerializationSchemaTest.java | 219 +++++++++++++++ .../sink/OceanBaseMetadataApplierTest.java | 256 ++++++++++++++++++ .../utils/OceanBaseTestMySQLCatalog.java | 128 +++++++++ .../oceanbase/OceanBaseTestUtils.java | 2 +- 15 files changed, 1010 insertions(+), 165 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/ListUtils.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java create mode 100644 flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java diff --git a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md index a41d93d95b8..4cbd737d82c 100644 --- a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md +++ b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md @@ -178,62 +178,6 @@ pipeline: Boolean Whether to enable partition calculation and flush records by partitions. Only works when 'sync-write' and 'direct-load.enabled' are 'false'. - - direct-load.enabled - No - false - Boolean - Whether to enable direct load. Note that direct load task requires the sink parallelism to be 1. - - - direct-load.host - No - - String - The hostname or IP address used in direct load task. Required when 'direct-load.enabled' is true. - - - direct-load.port - No - 2882 - Integer - The rpc port used in direct load task. Required when 'direct-load.enabled' is true. - - - direct-load.parallel - No - 8 - Integer - Parallelism of direct load task. - - - direct-load.max-error-rows - No - 0 - Long - Maximum tolerable number of error rows of direct load task. - - - direct-load.dup-action - No - REPLACE - String - Action when there is duplicated record of direct load task. Can be 'STOP_ON_DUP', 'REPLACE' or 'IGNORE'. - - - direct-load.timeout - No - 7d - Duration - Timeout for direct load task. - - - direct-load.heartbeat-timeout - No - 30s - Duration - Client heartbeat timeout in direct load task. - diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml index f0ae2b55429..2898c58b536 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/pom.xml @@ -64,6 +64,27 @@ limitations under the License. ${project.version} test + + org.apache.flink + flink-connector-oceanbase-cdc + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java index 11aacf09f7a..67828e66313 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseCatalog.java @@ -17,6 +17,8 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; +import org.apache.flink.cdc.common.types.DataType; + import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import com.oceanbase.connector.flink.connection.OceanBaseConnectionProvider; import org.apache.commons.compress.utils.Lists; @@ -37,7 +39,7 @@ public abstract class OceanBaseCatalog implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(OceanBaseCatalog.class); - private OceanBaseConnectionProvider connectionProvider; + protected OceanBaseConnectionProvider connectionProvider; private final OceanBaseConnectorOptions connectorOptions; public OceanBaseCatalog(OceanBaseConnectorOptions connectorOptions) { @@ -76,15 +78,28 @@ protected void executeUpdateStatement(String sql) throws SQLException { public abstract void createDatabase(String databaseName, boolean ignoreIfExists) throws OceanBaseCatalogException; + public abstract boolean tableExists(String databaseName, String tableName) + throws OceanBaseCatalogException; + public abstract void createTable(OceanBaseTable table, boolean ignoreIfExists) throws OceanBaseCatalogException; public abstract void alterAddColumns( String databaseName, String tableName, List addColumns); + public abstract void alterDropColumns( + String schemaName, String tableName, List dropColumns); + + public abstract void alterColumnType( + String schemaName, String tableName, String columnName, DataType dataType); + public abstract void renameColumn( String schemaName, String tableName, String oldColumnName, String newColumnName); + public abstract void dropTable(String schemaName, String tableName); + + public abstract void truncateTable(String schemaName, String tableName); + public void close() { LOG.info("Close OceanBase catalog"); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java index 9c821a3cd64..dea7edc9418 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseColumn.java @@ -46,17 +46,16 @@ public class OceanBaseColumn implements Serializable { @Nullable private final String defaultValue; /** - * The column size. COLUMN_SIZE in information_schema.COLUMNS. For numeric data, this is the - * maximum precision. For character data, this is the length in characters. For other data - * types, this is null. + * The column size. For numeric data, this is the maximum precision. For character data, this is + * the length in characters. For other data types, this is null. */ @Nullable private final Integer columnSize; /** * The number of fractional digits for numeric data. This is null for other data types. - * DECIMAL_DIGITS in information_schema.COLUMNS. + * NUMBER_SCALE in information_schema.COLUMNS. */ - @Nullable private final Integer decimalDigits; + @Nullable private final Integer numericScale; /** The column comment. COLUMN_COMMENT in information_schema.COLUMNS. */ @Nullable private final String columnComment; @@ -68,7 +67,7 @@ private OceanBaseColumn( boolean isNullable, @Nullable String defaultValue, @Nullable Integer columnSize, - @Nullable Integer decimalDigits, + @Nullable Integer numericScale, @Nullable String columnComment) { this.columnName = checkNotNull(columnName); this.ordinalPosition = ordinalPosition; @@ -76,7 +75,7 @@ private OceanBaseColumn( this.isNullable = isNullable; this.defaultValue = defaultValue; this.columnSize = columnSize; - this.decimalDigits = decimalDigits; + this.numericScale = numericScale; this.columnComment = columnComment; } @@ -104,8 +103,8 @@ public Optional getColumnSize() { return Optional.ofNullable(columnSize); } - public Optional getDecimalDigits() { - return Optional.ofNullable(decimalDigits); + public Optional getNumericScale() { + return Optional.ofNullable(numericScale); } public Optional getColumnComment() { @@ -130,8 +129,8 @@ public String toString() { + '\'' + ", columnSize=" + columnSize - + ", decimalDigits=" - + decimalDigits + + ", numericScale=" + + numericScale + ", columnComment='" + columnComment + '\'' @@ -153,7 +152,7 @@ public boolean equals(Object o) { && dataType.equalsIgnoreCase(column.dataType) && Objects.equals(defaultValue, column.defaultValue) && Objects.equals(columnSize, column.columnSize) - && Objects.equals(decimalDigits, column.decimalDigits) + && Objects.equals(numericScale, column.numericScale) && Objects.equals(columnComment, column.columnComment); } @@ -166,7 +165,7 @@ public static class Builder { private boolean isNullable = true; private String defaultValue; private Integer columnSize; - private Integer decimalDigits; + private Integer numericScale; private String columnComment; public OceanBaseColumn.Builder setColumnName(String columnName) { @@ -199,8 +198,8 @@ public OceanBaseColumn.Builder setColumnSize(Integer columnSize) { return this; } - public OceanBaseColumn.Builder setDecimalDigits(Integer decimalDigits) { - this.decimalDigits = decimalDigits; + public OceanBaseColumn.Builder setNumericScale(Integer numericScale) { + this.numericScale = numericScale; return this; } @@ -217,7 +216,7 @@ public OceanBaseColumn build() { isNullable, defaultValue, columnSize, - decimalDigits, + numericScale, columnComment); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index 3bdda647faf..63120311aa2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -17,6 +17,8 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.connectors.oceanbase.sink.OceanBaseUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -35,6 +37,8 @@ public class OceanBaseMySQLCatalog extends OceanBaseCatalog { private static final String RENAME_DDL = "ALTER TABLE `%s`.`%s` RENAME COLUMN `%s` TO `%s`"; + private static final String ALTER_COLUMN_TYPE_DDL = + "ALTER TABLE `%s`.`%s` MODIFY COLUMN `%s` %s;"; private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLCatalog.class); @@ -100,6 +104,29 @@ public void createDatabase(String databaseName, boolean ignoreIfExists) } } + @Override + public boolean tableExists(String databaseName, String tableName) + throws OceanBaseCatalogException { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + String querySql = + String.format( + "SELECT `TABLE_NAME` FROM `INFORMATION_SCHEMA`.`TABLES` WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME = '%s';", + databaseName, tableName); + try { + List dbList = executeSingleColumnStatement(querySql); + return !dbList.isEmpty(); + } catch (Exception e) { + LOG.error("Failed to check table exist, table: {}, sql: {}", tableName, querySql, e); + throw new OceanBaseCatalogException( + String.format("Failed to check table exist, table: %s", tableName), e); + } + } + /** * Creates a table. * @@ -166,6 +193,88 @@ public void alterAddColumns( } } + @Override + public void alterDropColumns(String databaseName, String tableName, List dropColumns) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + Preconditions.checkArgument(!dropColumns.isEmpty(), "Drop columns should not be empty."); + + String alterSql = buildAlterDropColumnsSql(databaseName, tableName, dropColumns); + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(alterSql); + LOG.info( + "Success to drop columns from {}.{}, duration: {}ms, sql: {}", + databaseName, + tableName, + System.currentTimeMillis() - startTimeMillis, + alterSql); + } catch (Exception e) { + LOG.error( + "Failed to drop columns from {}.{}, sql: {}", + databaseName, + tableName, + alterSql); + throw new OceanBaseCatalogException( + String.format("Failed to drop columns from %s.%s ", databaseName, tableName), + e); + } + } + + @Override + public void alterColumnType( + String databaseName, String tableName, String columnName, DataType dataType) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + + OceanBaseUtils.CdcDataTypeTransformer cdcDataTypeTransformer = + new OceanBaseUtils.CdcDataTypeTransformer(false, new OceanBaseColumn.Builder()); + OceanBaseColumn oceanBaseColumn = + dataType.accept(cdcDataTypeTransformer).setColumnName(columnName).build(); + String alterTypeSql = + String.format( + ALTER_COLUMN_TYPE_DDL, + databaseName, + tableName, + columnName, + oceanBaseColumn.getDataType()); + + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(alterTypeSql); + LOG.info( + "Success to alter table {}.{} column {} to type {}, duration: {}ms, sql: {}", + databaseName, + tableName, + columnName, + oceanBaseColumn.getDataType(), + System.currentTimeMillis() - startTimeMillis, + alterTypeSql); + } catch (Exception e) { + LOG.error( + "Failed to alter table {}.{} column {} to type {}, sql: {}", + databaseName, + tableName, + columnName, + oceanBaseColumn.getDataType(), + alterTypeSql, + e); + throw new OceanBaseCatalogException( + String.format( + "Failed to alter table %s.%s column %s to type %s ", + databaseName, tableName, columnName, oceanBaseColumn.getDataType()), + e); + } + } + @Override public void renameColumn( String schemaName, String tableName, String oldColumnName, String newColumnName) { @@ -197,6 +306,69 @@ public void renameColumn( } } + @Override + public void dropTable(String databaseName, String tableName) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + + String dropTableDDL = + String.format("DROP TABLE IF EXISTS `%s`.`%s`", databaseName, tableName); + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(dropTableDDL); + LOG.info( + "Success to drop table {}.{}, duration: {}ms, sql: {}", + databaseName, + tableName, + System.currentTimeMillis() - startTimeMillis, + dropTableDDL); + } catch (Exception e) { + LOG.error( + "Failed to drop table {}.{}, sql: {}", + databaseName, + tableName, + dropTableDDL, + e); + throw new OceanBaseCatalogException( + String.format("Failed to drop table %s.%s ", databaseName, tableName), e); + } + } + + @Override + public void truncateTable(String databaseName, String tableName) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + + String dropTableDDL = String.format("TRUNCATE TABLE `%s`.`%s`", databaseName, tableName); + try { + long startTimeMillis = System.currentTimeMillis(); + executeUpdateStatement(dropTableDDL); + LOG.info( + "Success to truncate table {}.{}, duration: {}ms, sql: {}", + databaseName, + tableName, + System.currentTimeMillis() - startTimeMillis, + dropTableDDL); + } catch (Exception e) { + LOG.error( + "Failed to truncate table {}.{}, sql: {}", + databaseName, + tableName, + dropTableDDL, + e); + throw new OceanBaseCatalogException( + String.format("Failed to truncate table %s.%s ", databaseName, tableName), e); + } + } + // ------------------------------------------------------------------------------------------ // OceanBase DDL SQL // ------------------------------------------------------------------------------------------ @@ -246,6 +418,19 @@ protected String buildCreateTableSql(OceanBaseTable table, boolean ignoreIfExist return builder.toString(); } + private String buildAlterDropColumnsSql( + String databaseName, String tableName, List dropColumns) { + StringBuilder builder = new StringBuilder(); + builder.append(String.format("ALTER TABLE `%s`.`%s` ", databaseName, tableName)); + String columnsStmt = + dropColumns.stream() + .map(col -> String.format("DROP COLUMN `%s`", col)) + .collect(Collectors.joining(", ")); + builder.append(columnsStmt); + builder.append(";"); + return builder.toString(); + } + protected String buildColumnStmt(OceanBaseColumn column) { StringBuilder builder = new StringBuilder(); builder.append("`"); @@ -253,7 +438,7 @@ protected String buildColumnStmt(OceanBaseColumn column) { builder.append("` "); builder.append( getFullColumnType( - column.getDataType(), column.getColumnSize(), column.getDecimalDigits())); + column.getDataType(), column.getColumnSize(), column.getNumericScale())); builder.append(" "); builder.append(column.isNullable() ? "NULL" : "NOT NULL"); if (column.getDefaultValue().isPresent()) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java index afb6e112e74..c77236988f6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseOracleCatalog.java @@ -17,6 +17,8 @@ package org.apache.flink.cdc.connectors.oceanbase.catalog; +import org.apache.flink.cdc.common.types.DataType; + import com.oceanbase.connector.flink.OceanBaseConnectorOptions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +50,13 @@ public void createDatabase(String databaseName, boolean ignoreIfExists) "This operation under oracle tenant is not supported currently."); } + @Override + public boolean tableExists(String databaseName, String tableName) + throws OceanBaseCatalogException { + throw new OceanBaseCatalogException( + "This operation under oracle tenant is not supported currently."); + } + @Override public void createTable(OceanBaseTable table, boolean ignoreIfExists) throws OceanBaseCatalogException { @@ -62,10 +71,35 @@ public void alterAddColumns( "This operation under oracle tenant is not supported currently."); } + @Override + public void alterDropColumns(String schemaName, String tableName, List dropColumns) { + throw new OceanBaseCatalogException( + "This operation under oracle tenant is not supported currently."); + } + + @Override + public void alterColumnType( + String schemaName, String tableName, String columnName, DataType dataType) { + throw new OceanBaseCatalogException( + "This operation under oracle tenant is not supported currently."); + } + @Override public void renameColumn( String schemaName, String tableName, String oldColumnName, String newColumnName) { throw new OceanBaseCatalogException( "This operation under oracle tenant is not supported currently."); } + + @Override + public void dropTable(String schemaName, String tableName) { + throw new OceanBaseCatalogException( + "This operation under oracle tenant is not supported currently."); + } + + @Override + public void truncateTable(String schemaName, String tableName) { + throw new OceanBaseCatalogException( + "This operation under oracle tenant is not supported currently."); + } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java index ca82835c54e..b206d8eb9da 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/factory/OceanBaseDataSinkFactory.java @@ -79,14 +79,6 @@ public Set> optionalOptions() { optionalOptions.add(OceanBaseDataSinkOptions.MEMSTORE_THRESHOLD); optionalOptions.add(OceanBaseDataSinkOptions.MEMSTORE_CHECK_INTERVAL); optionalOptions.add(OceanBaseDataSinkOptions.PARTITION_ENABLED); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_ENABLED); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_HOST); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_PORT); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_PARALLEL); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_MAX_ERROR_ROWS); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_DUP_ACTION); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_TIMEOUT); - optionalOptions.add(OceanBaseDataSinkOptions.DIRECT_LOAD_HEARTBEAT_TIMEOUT); optionalOptions.add(OceanBaseDataSinkOptions.SYNC_WRITE); optionalOptions.add(OceanBaseDataSinkOptions.BUFFER_FLUSH_INTERVAL); optionalOptions.add(OceanBaseDataSinkOptions.BUFFER_SIZE); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java index dfdcaf22ce2..aa04af98862 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseDataSinkOptions.java @@ -17,12 +17,9 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; -import org.apache.flink.cdc.common.annotation.Experimental; import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.ConfigOptions; -import com.alipay.oceanbase.rpc.protocol.payload.impl.direct_load.ObLoadDupActionType; - import java.time.Duration; /** Options for {@link OceanBaseDataSink}. */ @@ -88,62 +85,6 @@ public class OceanBaseDataSinkOptions { .withDescription( "Whether to enable partition calculation and flush records by partitions. Default value is 'false'."); - @Experimental - public static final ConfigOption DIRECT_LOAD_ENABLED = - ConfigOptions.key("direct-load.enabled") - .booleanType() - .defaultValue(false) - .withDescription("Whether to enable direct load."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_HOST = - ConfigOptions.key("direct-load.host") - .stringType() - .noDefaultValue() - .withDescription("Hostname used in direct load."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_PORT = - ConfigOptions.key("direct-load.port") - .intType() - .defaultValue(2882) - .withDescription("Rpc port number used in direct load."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_PARALLEL = - ConfigOptions.key("direct-load.parallel") - .intType() - .defaultValue(8) - .withDescription("Parallelism of direct load."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_MAX_ERROR_ROWS = - ConfigOptions.key("direct-load.max-error-rows") - .longType() - .defaultValue(0L) - .withDescription("Maximum tolerable number of error rows."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_DUP_ACTION = - ConfigOptions.key("direct-load.dup-action") - .enumType(ObLoadDupActionType.class) - .defaultValue(ObLoadDupActionType.REPLACE) - .withDescription("Action when there is duplicated record in direct load."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_TIMEOUT = - ConfigOptions.key("direct-load.timeout") - .durationType() - .defaultValue(Duration.ofDays(7)) - .withDescription("Timeout for direct load task."); - - @Experimental - public static final ConfigOption DIRECT_LOAD_HEARTBEAT_TIMEOUT = - ConfigOptions.key("direct-load.heartbeat-timeout") - .durationType() - .defaultValue(Duration.ofSeconds(30)) - .withDescription("Client heartbeat timeout in direct load task."); - public static final ConfigOption SYNC_WRITE = ConfigOptions.key("sync-write") .booleanType() diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java index 8ada243115e..65aa715194f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplier.java @@ -21,12 +21,17 @@ import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; +import org.apache.flink.cdc.common.event.visitor.SchemaChangeEventVisitor; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalog; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; @@ -60,22 +65,36 @@ public OceanBaseMetadataApplier(OceanBaseConnectorOptions connectorOptions) { @Override public void applySchemaChange(SchemaChangeEvent event) { - try { - if (event instanceof CreateTableEvent) { - applyCreateTableEvent((CreateTableEvent) event); - } else if (event instanceof AddColumnEvent) { - applyAddColumnEvent((AddColumnEvent) event); - } else if (event instanceof DropColumnEvent) { - applyDropColumnEvent((DropColumnEvent) event); - } else if (event instanceof RenameColumnEvent) { - applyRenameColumnEvent((RenameColumnEvent) event); - } else if (event instanceof AlterColumnTypeEvent) { - throw new RuntimeException("Unsupported schema change event, " + event); - } - } catch (Exception ex) { - throw new RuntimeException( - "Failed to schema change, " + event + ", reason: " + ex.getMessage()); - } + SchemaChangeEventVisitor.visit( + event, + addColumnEvent -> { + applyAddColumnEvent(addColumnEvent); + return null; + }, + alterColumnTypeEvent -> { + applyAlterColumnTypeEvent(alterColumnTypeEvent); + return null; + }, + createTableEvent -> { + applyCreateTableEvent(createTableEvent); + return null; + }, + dropColumnEvent -> { + applyDropColumnEvent(dropColumnEvent); + return null; + }, + dropTableEvent -> { + applyDropTableEvent(dropTableEvent); + return null; + }, + renameColumnEvent -> { + applyRenameColumnEvent(renameColumnEvent); + return null; + }, + truncateTableEvent -> { + applyTruncateTableEvent(truncateTableEvent); + return null; + }); } private void applyCreateTableEvent(CreateTableEvent createTableEvent) @@ -123,9 +142,24 @@ private void applyAddColumnEvent(AddColumnEvent addColumnEvent) { } private void applyDropColumnEvent(DropColumnEvent dropColumnEvent) { - // TODO The `DropColumnEvent` in OceanBase is classified as an offline DDL operation, - // and currently, this pipeline connector does not support offline DDL actions. - throw new UnsupportedOperationException("Drop column is not supported currently"); + List dropColumns = dropColumnEvent.getDroppedColumnNames(); + catalog.alterDropColumns( + dropColumnEvent.tableId().getSchemaName(), + dropColumnEvent.tableId().getTableName(), + dropColumns); + } + + private void applyAlterColumnTypeEvent(AlterColumnTypeEvent alterColumnTypeEvent) { + TableId tableId = alterColumnTypeEvent.tableId(); + Map typeMapping = alterColumnTypeEvent.getTypeMapping(); + + for (Map.Entry entry : typeMapping.entrySet()) { + catalog.alterColumnType( + tableId.getSchemaName(), + tableId.getTableName(), + entry.getKey(), + entry.getValue()); + } } private void applyRenameColumnEvent(RenameColumnEvent renameColumnEvent) { @@ -139,4 +173,23 @@ private void applyRenameColumnEvent(RenameColumnEvent renameColumnEvent) { entry.getValue()); } } + + private void applyDropTableEvent(DropTableEvent dropTableEvent) { + TableId tableId = dropTableEvent.tableId(); + catalog.dropTable(tableId.getSchemaName(), tableId.getTableName()); + } + + private void applyTruncateTableEvent(TruncateTableEvent truncateTableEvent) { + TableId tableId = truncateTableEvent.tableId(); + + // check table exists + if (catalog.tableExists(tableId.getSchemaName(), tableId.getTableName())) { + catalog.truncateTable(tableId.getSchemaName(), tableId.getTableName()); + } else { + throw new OceanBaseCatalogException( + String.format( + "Failed to truncate table %s.%s, because the table not exist", + tableId.getSchemaName(), tableId.getTableName())); + } + } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index e36c9913ad8..d8ff517302a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -35,8 +35,10 @@ import org.apache.flink.cdc.common.types.TimestampType; import org.apache.flink.cdc.common.types.TinyIntType; import org.apache.flink.cdc.common.types.VarCharType; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; +import org.apache.flink.cdc.connectors.oceanbase.utils.ListUtils; import org.apache.commons.collections.CollectionUtils; @@ -51,15 +53,26 @@ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { List columns = schema.getColumns(); List primaryKeys = schema.primaryKeys(); + List partitionKeys = schema.partitionKeys(); List oceanBaseColumns = new ArrayList<>(); for (int i = 0; i < columns.size(); i++) { Column column = columns.get(i); + // Since OceanBase partition tables are not yet supported, we first use + // partitionKeys as part of primaryKeys. + boolean isPrimaryKeys = + primaryKeys.contains(column.getName()) + || partitionKeys.contains(column.getName()); + // All parts of a PRIMARY KEY must be NOT NULL. if you need NULL in a key, use UNIQUE + // instead. + checkPrimaryKeyNotNull(isPrimaryKeys, column.getType().isNullable()); + OceanBaseColumn.Builder builder = new OceanBaseColumn.Builder() .setColumnName(column.getName()) .setOrdinalPosition(i) + .setDefaultValue(column.getDefaultValueExpression()) .setColumnComment(column.getComment()); - toOceanBaseDataType(column, primaryKeys.contains(column.getName()), builder); + toOceanBaseDataType(column, isPrimaryKeys, builder); oceanBaseColumns.add(builder.build()); } @@ -68,11 +81,13 @@ public static OceanBaseTable toOceanBaseTable(TableId tableId, Schema schema) { .setDatabaseName(tableId.getSchemaName()) .setTableName(tableId.getTableName()) .setTableType( - CollectionUtils.isNotEmpty(schema.primaryKeys()) + CollectionUtils.isNotEmpty(primaryKeys) ? OceanBaseTable.TableType.PRIMARY_KEY : OceanBaseTable.TableType.DUPLICATE_KEY) .setColumns(oceanBaseColumns) - .setTableKeys(schema.primaryKeys()) + // Since OceanBase partition tables are not yet supported, we first use + // partitionKeys as part of primaryKeys. + .setTableKeys(ListUtils.union(primaryKeys, partitionKeys)) .setPartitionKeys(schema.partitionKeys()) .setComment(schema.comment()); return tableBuilder.build(); @@ -179,7 +194,7 @@ public OceanBaseColumn.Builder visit(DoubleType doubleType) { public OceanBaseColumn.Builder visit(DecimalType decimalType) { builder.setDataType(DECIMAL); builder.setColumnSize(decimalType.getPrecision()); - builder.setDecimalDigits(decimalType.getScale()); + builder.setNumericScale(decimalType.getScale()); builder.setNullable(decimalType.isNullable()); return builder; } @@ -209,6 +224,8 @@ public OceanBaseColumn.Builder visit(VarCharType varCharType) { // case for string type to avoid row size too large if (varCharType.getLength() == VarCharType.MAX_LENGTH) { builder.setDataType(TEXT); + // A text column can't have a default value in OceanBase. + builder.setDefaultValue(null); } return builder; } @@ -241,4 +258,12 @@ protected OceanBaseColumn.Builder defaultMethod(DataType dataType) { throw new UnsupportedOperationException("Unsupported CDC data type " + dataType); } } + + private static void checkPrimaryKeyNotNull(boolean isPrimaryKeys, boolean dataTypeNullable) { + if (isPrimaryKeys && dataTypeNullable) { + throw new OceanBaseCatalogException( + "All parts of a PRIMARY KEY must be NOT NULL in OceanBase. " + + "if you need NULL in a key, use UNIQUE instead."); + } + } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/ListUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/ListUtils.java new file mode 100644 index 00000000000..be7efe5722b --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/ListUtils.java @@ -0,0 +1,33 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.utils; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** A utility class for {@link List}. */ +public class ListUtils { + + public static List union(List listA, List listB) { + Set set = new HashSet<>(listA); + set.addAll(listB); + return new ArrayList<>(set); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java new file mode 100644 index 00000000000..c8c0a395e05 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java @@ -0,0 +1,219 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.data.DecimalData; +import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimestampData; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.BooleanType; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DateType; +import org.apache.flink.cdc.common.types.DecimalType; +import org.apache.flink.cdc.common.types.FloatType; +import org.apache.flink.cdc.common.types.IntType; +import org.apache.flink.cdc.common.types.LocalZonedTimestampType; +import org.apache.flink.cdc.common.types.SmallIntType; +import org.apache.flink.cdc.common.types.TimestampType; +import org.apache.flink.cdc.common.types.VarCharType; +import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; + +import com.oceanbase.connector.flink.table.Record; +import org.junit.Test; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.Objects; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +/** Tests for {@link OceanBaseEventSerializationSchema}. */ +public class OceanBaseEventSerializationSchemaTest { + + private static final OceanBaseEventSerializationSchema serializer = + new OceanBaseEventSerializationSchema(ZoneId.of("+08")); + + @Test + public void testMixedSchemaAndDataChanges() throws Exception { + // 1. create table1, and insert/delete/update data + TableId table1 = TableId.parse("test.tbl1"); + Schema schema1 = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new TimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent1 = new CreateTableEvent(table1, schema1); + assertNull(serializer.serialize(createTableEvent1)); + + BinaryRecordDataGenerator generator1 = + new BinaryRecordDataGenerator( + schema1.getColumnDataTypes().toArray(new DataType[0])); + DataChangeEvent insertEvent1 = + DataChangeEvent.insertEvent( + table1, + generator1.generate( + new Object[] { + 1, + true, + TimestampData.fromTimestamp( + Timestamp.valueOf("2023-11-27 18:00:00")) + })); + Record insert1 = serializer.serialize(insertEvent1); + verifySerializeResult(table1, "[1, true, 2023-11-27 18:00:00.0]", insert1); + + DataChangeEvent deleteEvent1 = + DataChangeEvent.deleteEvent( + table1, + generator1.generate( + new Object[] { + 2, + false, + TimestampData.fromTimestamp( + Timestamp.valueOf("2023-11-27 19:00:00")) + })); + verifySerializeResult( + table1, "[2, false, 2023-11-27 19:00:00.0]", serializer.serialize(deleteEvent1)); + + DataChangeEvent updateEvent1 = + DataChangeEvent.updateEvent( + table1, + generator1.generate( + new Object[] { + 3, + false, + TimestampData.fromTimestamp( + Timestamp.valueOf("2023-11-27 20:00:00")) + }), + generator1.generate( + new Object[] { + 3, + true, + TimestampData.fromTimestamp( + Timestamp.valueOf("2023-11-27 21:00:00")) + })); + verifySerializeResult( + table1, "[3, true, 2023-11-27 21:00:00.0]", serializer.serialize(updateEvent1)); + + // 2. create table2, and insert data + TableId table2 = TableId.parse("test.tbl2"); + Schema schema2 = + Schema.newBuilder() + .physicalColumn("col1", new DateType()) + .physicalColumn("col2", new FloatType()) + .physicalColumn("col3", new VarCharType(20)) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent2 = new CreateTableEvent(table2, schema2); + assertNull(serializer.serialize(createTableEvent2)); + + BinaryRecordDataGenerator generator2 = + new BinaryRecordDataGenerator( + schema2.getColumnDataTypes().toArray(new DataType[0])); + DataChangeEvent insertEvent2 = + DataChangeEvent.insertEvent( + table2, + generator2.generate( + new Object[] { + (int) LocalDate.of(2023, 11, 27).toEpochDay(), + 3.4f, + BinaryStringData.fromString("insert table2") + })); + verifySerializeResult( + table2, "[2023-11-27, 3.4, insert table2]", serializer.serialize(insertEvent2)); + + // 3. add columns to table1, and delete data + AddColumnEvent addColumnEvent = + new AddColumnEvent( + table1, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("col4", new DecimalType(20, 5))), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("col5", new SmallIntType())), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "col6", new LocalZonedTimestampType())))); + Schema newSchema1 = SchemaUtils.applySchemaChangeEvent(schema1, addColumnEvent); + BinaryRecordDataGenerator newGenerator1 = + new BinaryRecordDataGenerator( + newSchema1.getColumnDataTypes().toArray(new DataType[0])); + assertNull(serializer.serialize(addColumnEvent)); + + DataChangeEvent deleteEvent2 = + DataChangeEvent.deleteEvent( + table1, + newGenerator1.generate( + new Object[] { + 4, + true, + TimestampData.fromTimestamp( + Timestamp.valueOf("2023-11-27 21:00:00")), + DecimalData.fromBigDecimal(new BigDecimal("83.23"), 20, 5), + (short) 9, + LocalZonedTimestampData.fromInstant( + LocalDateTime.of(2023, 11, 27, 21, 0, 0) + .toInstant(ZoneOffset.of("+10"))) + })); + verifySerializeResult( + table1, + "[4, true, 2023-11-27 21:00:00.0, 83.23000, 9, 2023-11-27 19:00:00.0]", + Objects.requireNonNull(serializer.serialize(deleteEvent2))); + + // 4. drop columns from table2, and insert data + DropColumnEvent dropColumnEvent = + new DropColumnEvent(table2, Arrays.asList("col2", "col3")); + Schema newSchema2 = SchemaUtils.applySchemaChangeEvent(schema2, dropColumnEvent); + BinaryRecordDataGenerator newGenerator2 = + new BinaryRecordDataGenerator( + newSchema2.getColumnDataTypes().toArray(new DataType[0])); + assertNull(serializer.serialize(dropColumnEvent)); + + DataChangeEvent insertEvent3 = + DataChangeEvent.insertEvent( + table2, + newGenerator2.generate( + new Object[] {(int) LocalDate.of(2023, 11, 28).toEpochDay()})); + verifySerializeResult( + table2, "[2023-11-28]", Objects.requireNonNull(serializer.serialize(insertEvent3))); + } + + private void verifySerializeResult(TableId expectTable, String expectRow, Record record) + throws Exception { + assertEquals(expectTable.getSchemaName(), record.getTableId().getSchemaName()); + assertEquals(expectTable.getTableName(), record.getTableId().getTableName()); + int start = record.toString().indexOf('['); + int end = record.toString().indexOf(']'); + assertEquals(expectRow, record.toString().substring(start, end + 1)); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java new file mode 100644 index 00000000000..42121778b14 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java @@ -0,0 +1,256 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.sink; + +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.common.types.BooleanType; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.IntType; +import org.apache.flink.cdc.common.types.LocalZonedTimestampType; +import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; +import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; +import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseTestMySQLCatalog; + +import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseMySQLModeITCase.NETWORK; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** Tests for {@link OceanBaseMetadataApplier}. */ +public class OceanBaseMetadataApplierTest { + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMetadataApplierTest.class); + + private OceanBaseMetadataApplier metadataApplier; + private OceanBaseTestMySQLCatalog catalog; + + @ClassRule + public static final OceanBaseContainer OB_SERVER = + OceanBaseTestUtils.createOceanBaseContainerForJdbc() + .withNetwork(NETWORK) + .withNetworkAliases("oceanbase") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @Before + public void setup() throws Exception { + final ImmutableMap configMap = + ImmutableMap.builder() + .put("url", OB_SERVER.getJdbcUrl()) + .put("username", OB_SERVER.getUsername()) + .put("password", OB_SERVER.getPassword()) + .build(); + OceanBaseConnectorOptions connectorOptions = new OceanBaseConnectorOptions(configMap); + metadataApplier = new OceanBaseMetadataApplier(connectorOptions); + catalog = new OceanBaseTestMySQLCatalog(connectorOptions); + catalog.open(); + } + + @AfterClass + public static void close() { + OB_SERVER.close(); + } + + @Test + public void testCreateTable() { + TableId tableId = TableId.parse("test.tbl1"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new LocalZonedTimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + List columns = new ArrayList<>(); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col1") + .setOrdinalPosition(0) + .setDataType("int") + .setNumericScale(0) + .setNullable(false) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col2") + .setOrdinalPosition(1) + .setDataType("tinyint") + .setNumericScale(0) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col3") + .setOrdinalPosition(2) + .setDataType("datetime") + .setNullable(true) + .build()); + OceanBaseTable expectTable = + new OceanBaseTable.Builder() + .setDatabaseName(tableId.getSchemaName()) + .setTableName(tableId.getTableName()) + .setTableType(OceanBaseTable.TableType.PRIMARY_KEY) + .setColumns(columns) + .setTableKeys(schema.primaryKeys()) + .build(); + + System.out.println(expectTable); + System.out.println(actualTable); + assertEquals(expectTable, actualTable); + } + + @Test + public void testDropTable() { + TableId tableId = TableId.parse("test.tbl2"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new LocalZonedTimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + DropTableEvent dropTableEvent = new DropTableEvent(tableId); + metadataApplier.applySchemaChange(dropTableEvent); + OceanBaseTable actualDropTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + Assert.assertNull(actualDropTable); + } + + @Test + public void testTruncateTable() throws SQLException { + TableId tableId = TableId.parse("test.tbl3"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new LocalZonedTimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + String insertSQL = + String.format("insert into %s values(1, true, now())", tableId.identifier()); + catalog.executeUpdateStatement(insertSQL); + + TruncateTableEvent dropTableEvent = new TruncateTableEvent(tableId); + metadataApplier.applySchemaChange(dropTableEvent); + + List values = + catalog.executeSingleColumnStatement( + String.format("select col1 from %s", tableId.identifier())); + Assert.assertTrue(values.isEmpty()); + } + + @Test + public void testDropColumnEvent() { + TableId tableId = TableId.parse("test.tbl4"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new LocalZonedTimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + DropColumnEvent dropColumnEvent = new DropColumnEvent(tableId, Lists.newArrayList("col2")); + metadataApplier.applySchemaChange(dropColumnEvent); + + OceanBaseTable changeTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + + Assert.assertNotEquals(actualTable.getColumns().size(), changeTable.getColumns().size()); + } + + @Test + public void testAlterColumnType() { + TableId tableId = TableId.parse("test.tbl5"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", new BooleanType()) + .physicalColumn("col3", new LocalZonedTimestampType()) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + ImmutableMap map = ImmutableMap.of("col2", new IntType()); + AlterColumnTypeEvent alterColumnTypeEvent = new AlterColumnTypeEvent(tableId, map); + metadataApplier.applySchemaChange(alterColumnTypeEvent); + + OceanBaseTable changeTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + + System.out.println(actualTable); + System.out.println(changeTable); + Assert.assertNotEquals( + actualTable.getColumn("col2").getDataType(), + changeTable.getColumn("col2").getDataType()); + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java new file mode 100644 index 00000000000..3abf1923dfa --- /dev/null +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java @@ -0,0 +1,128 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.connectors.oceanbase.utils; + +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseMySQLCatalog; +import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import com.oceanbase.connector.flink.OceanBaseConnectorOptions; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/** A {@link OceanBaseMySQLCatalog} only for test. */ +public class OceanBaseTestMySQLCatalog extends OceanBaseMySQLCatalog { + public OceanBaseTestMySQLCatalog(OceanBaseConnectorOptions connectorOptions) { + super(connectorOptions); + super.open(); + } + + public Optional getTable(String databaseName, String tableName) { + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "database name cannot be null or empty."); + Preconditions.checkArgument( + !StringUtils.isNullOrWhitespaceOnly(tableName), + "table name cannot be null or empty."); + + final String tableSchemaQuery = + "SELECT `COLUMN_NAME`, `DATA_TYPE`, `ORDINAL_POSITION`, `NUMERIC_SCALE`, " + + "`IS_NULLABLE`, `COLUMN_KEY`, `COLUMN_COMMENT` FROM `information_schema`.`COLUMNS` " + + "WHERE `TABLE_SCHEMA`=? AND `TABLE_NAME`=?;"; + + OceanBaseTable.TableType tableType = OceanBaseTable.TableType.UNKNOWN; + List columns = new ArrayList<>(); + List tableKeys = new ArrayList<>(); + try (Connection connection = connectionProvider.getConnection()) { + try (PreparedStatement statement = connection.prepareStatement(tableSchemaQuery)) { + statement.setObject(1, databaseName); + statement.setObject(2, tableName); + try (ResultSet resultSet = statement.executeQuery()) { + while (resultSet.next()) { + String name = resultSet.getString("COLUMN_NAME"); + String type = resultSet.getString("DATA_TYPE"); + int position = resultSet.getInt("ORDINAL_POSITION"); + Integer scale = resultSet.getInt("NUMERIC_SCALE"); + if (resultSet.wasNull()) { + scale = null; + } + String isNullable = resultSet.getString("IS_NULLABLE"); + String comment = resultSet.getString("COLUMN_COMMENT"); + OceanBaseColumn column = + new OceanBaseColumn.Builder() + .setColumnName(name) + .setOrdinalPosition(position - 1) + .setDataType(type) + .setNumericScale(scale) + .setNullable( + isNullable == null + || !isNullable.equalsIgnoreCase("NO")) + .setColumnComment(comment.isEmpty() ? null : comment) + .build(); + columns.add(column); + + // Only primary key table has value in this field. and the value is "PRI" + String columnKey = resultSet.getString("COLUMN_KEY"); + if (!StringUtils.isNullOrWhitespaceOnly(columnKey)) { + if (columnKey.equalsIgnoreCase("PRI") + && tableType == OceanBaseTable.TableType.UNKNOWN) { + tableType = OceanBaseTable.TableType.PRIMARY_KEY; + } + tableKeys.add(column.getColumnName()); + } + } + } + } + } catch (Exception e) { + throw new OceanBaseCatalogException( + String.format("Failed to get table %s.%s", databaseName, tableName), e); + } + + OceanBaseTable oceanBaseTable = null; + if (!columns.isEmpty()) { + oceanBaseTable = + new OceanBaseTable.Builder() + .setDatabaseName(databaseName) + .setTableName(tableName) + .setTableType(tableType) + .setColumns(columns) + .setTableKeys(tableKeys) + .build(); + } + return Optional.ofNullable(oceanBaseTable); + } + + @Override + public List executeSingleColumnStatement(String sql) throws SQLException { + return super.executeSingleColumnStatement(sql); + } + + @Override + public void executeUpdateStatement(String sql) throws SQLException { + super.executeUpdateStatement(sql); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java index a8eec4f859f..6a861c03e66 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java @@ -46,7 +46,7 @@ public static OceanBaseContainer createOceanBaseContainerForCDC() { public static OceanBaseContainer createOceanBaseContainerForJdbc() { return createOceanBaseContainer(LATEST_VERSION, "mini") - .withStartupTimeout(Duration.ofMinutes(2)); + .withStartupTimeout(Duration.ofMinutes(4)); } public static OceanBaseContainer createOceanBaseContainer(String version, String mode) { From 93e6d5e38b32425be42eec020272419f7f3e1bc3 Mon Sep 17 00:00:00 2001 From: yuanoOo Date: Wed, 20 Nov 2024 16:54:57 +0800 Subject: [PATCH 23/23] =?UTF-8?q?Enhancement=EF=BC=9AFix=20code.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../pipeline-connectors/oceanbase.md | 32 ++- .../catalog/OceanBaseMySQLCatalog.java | 2 + .../OceanBaseEventSerializationSchema.java | 29 ++- .../oceanbase/sink/OceanBaseRowConvert.java | 5 + .../oceanbase/sink/OceanBaseUtils.java | 69 +++++- .../sink/OceanBaseMetadataApplierTest.java | 210 +++++++++++++++++- .../utils/OceanBaseTestMySQLCatalog.java | 7 +- .../pipeline/tests/OceanBaseE2eITCase.java | 8 +- 8 files changed, 344 insertions(+), 18 deletions(-) diff --git a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md index 4cbd737d82c..19ccbcab4b6 100644 --- a/docs/content/docs/connectors/pipeline-connectors/oceanbase.md +++ b/docs/content/docs/connectors/pipeline-connectors/oceanbase.md @@ -230,6 +230,21 @@ pipeline: BIGINT + + BINARY + BINARY + + + + VARBINARY(n) when n <= 1048576 + VARBINARY(n) + + + + VARBINARY(n) when n > 1048576 + LONGBLOB + + FLOAT FLOAT @@ -255,8 +270,18 @@ pipeline: DATE + + TIME + TIME + + TIMESTAMP + DATETIME + + + + TIMESTAMP_TZ TIMESTAMP @@ -276,8 +301,13 @@ pipeline: + VARCHAR(n) where n <= 262144 VARCHAR(n) - VARCHAR(n) + + + + VARCHAR(n) where n > 262144 + TEXT diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java index 63120311aa2..900b0e050df 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/catalog/OceanBaseMySQLCatalog.java @@ -463,9 +463,11 @@ protected String getFullColumnType( return String.format("DECIMAL(%d, %s)", columnSize.get(), decimalDigits.get()); case "CHAR": case "VARCHAR": + case "VARBINARY": Preconditions.checkArgument( columnSize.isPresent(), type + " type must have column size"); return String.format("%s(%d)", dataType, columnSize.get()); + case "DATETIME": case "TIMESTAMP": return columnSize .map(size -> String.format("%s(%d)", dataType, size)) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java index b94bb20e6dd..fd966af34af 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchema.java @@ -29,6 +29,8 @@ import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.common.utils.SchemaUtils; +import org.apache.flink.shaded.guava31.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder; import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import com.oceanbase.connector.flink.table.DataChangeRecord; @@ -41,10 +43,14 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; /** A serializer for Event to Record. */ public class OceanBaseEventSerializationSchema implements RecordSerializationSchema { + private final Cache> cache = + CacheBuilder.newBuilder().build(); private final Map schemaMaps = new HashMap<>(); /** ZoneId from pipeline config to support timestamp with local time zone. */ @@ -128,11 +134,26 @@ public Object[] serializerRecord(RecordData recordData, Schema schema) { "Column size does not match the data size"); Object[] values = new Object[columns.size()]; + List converters = null; + try { + converters = + cache.get( + schema, + () -> + columns.stream() + .map( + column -> + OceanBaseRowConvert + .createNullableExternalConverter( + column.getType(), + pipelineZoneId)) + .collect(Collectors.toList())); + } catch (ExecutionException e) { + throw new RuntimeException("Failed to obtain SerializationConverter cache", e); + } + for (int i = 0; i < recordData.getArity(); i++) { - OceanBaseRowConvert.SerializationConverter converter = - OceanBaseRowConvert.createNullableExternalConverter( - columns.get(i).getType(), pipelineZoneId); - Object field = converter.serialize(i, recordData); + Object field = converters.get(i).serialize(i, recordData); values[i] = field; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java index b42d2fc1ed9..18adbb1c81d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java @@ -34,8 +34,10 @@ import java.io.IOException; import java.io.Serializable; import java.sql.Date; +import java.sql.Time; import java.sql.Timestamp; import java.time.LocalDate; +import java.time.LocalTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.HashMap; @@ -100,6 +102,9 @@ static SerializationConverter createExternalConverter(DataType type, ZoneId pipe return (index, val) -> val.getDouble(index); case DATE: return (index, val) -> Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))); + case TIME_WITHOUT_TIME_ZONE: + return (index, val) -> + Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000)); case TIMESTAMP_WITHOUT_TIME_ZONE: return (index, val) -> val.getTimestamp(index, DataTypeChecks.getPrecision(type)).toTimestamp(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java index d8ff517302a..dc1adc03e9e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.BigIntType; +import org.apache.flink.cdc.common.types.BinaryType; import org.apache.flink.cdc.common.types.BooleanType; import org.apache.flink.cdc.common.types.CharType; import org.apache.flink.cdc.common.types.DataType; @@ -32,9 +33,12 @@ import org.apache.flink.cdc.common.types.IntType; import org.apache.flink.cdc.common.types.LocalZonedTimestampType; import org.apache.flink.cdc.common.types.SmallIntType; +import org.apache.flink.cdc.common.types.TimeType; import org.apache.flink.cdc.common.types.TimestampType; import org.apache.flink.cdc.common.types.TinyIntType; +import org.apache.flink.cdc.common.types.VarBinaryType; import org.apache.flink.cdc.common.types.VarCharType; +import org.apache.flink.cdc.common.types.ZonedTimestampType; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseCatalogException; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseColumn; import org.apache.flink.cdc.connectors.oceanbase.catalog.OceanBaseTable; @@ -110,6 +114,8 @@ public static void toOceanBaseDataType( public static final String SMALLINT = "SMALLINT"; public static final String INT = "INT"; public static final String BIGINT = "BIGINT"; + public static final String BINARY = "BINARY"; + public static final String VARBINARY = "VARBINARY"; public static final String FLOAT = "FLOAT"; public static final String DOUBLE = "DOUBLE"; public static final String DECIMAL = "DECIMAL"; @@ -119,9 +125,10 @@ public static void toOceanBaseDataType( public static final String DATE = "DATE"; public static final String DATETIME = "DATETIME"; public static final String TIMESTAMP = "TIMESTAMP"; + public static final String TIME = "TIME"; public static final String JSON = "JSON"; public static final String TEXT = "TEXT"; - public static final String BLOB = "BLOB"; + public static final String LONGBLOB = "LONGBLOB"; /** Max size of char type of OceanBase. */ public static final int MAX_CHAR_SIZE = 256; @@ -129,6 +136,9 @@ public static void toOceanBaseDataType( /** Max size of varchar type of OceanBase. */ public static final int MAX_VARCHAR_SIZE = 262144; + /** The max VARBINARY column length is 1048576. */ + public static final int MAX_VARBINARY_SIZE = 1048576; + /** Transforms CDC {@link DataType} to OceanBase data type. */ public static class CdcDataTypeTransformer extends DataTypeDefaultVisitor { @@ -176,6 +186,30 @@ public OceanBaseColumn.Builder visit(BigIntType bigIntType) { return builder; } + @Override + public OceanBaseColumn.Builder visit(BinaryType binaryType) { + builder.setDataType(BINARY); + builder.setNullable(binaryType.isNullable()); + return builder; + } + + @Override + public OceanBaseColumn.Builder visit(VarBinaryType bytesType) { + int length = bytesType.getLength(); + + // The max VARBINARY column length is 1048576. + if (length <= MAX_VARBINARY_SIZE) { + builder.setDataType(VARBINARY); + builder.setColumnSize(length); + builder.setNullable(bytesType.isNullable()); + } else { + builder.setDataType(LONGBLOB); + builder.setNullable(bytesType.isNullable()); + } + + return builder; + } + @Override public OceanBaseColumn.Builder visit(FloatType floatType) { builder.setDataType(FLOAT); @@ -222,7 +256,7 @@ public OceanBaseColumn.Builder visit(VarCharType varCharType) { builder.setColumnSize(Math.min(length, MAX_VARCHAR_SIZE)); // case for string type to avoid row size too large - if (varCharType.getLength() == VarCharType.MAX_LENGTH) { + if (varCharType.getLength() > MAX_VARCHAR_SIZE) { builder.setDataType(TEXT); // A text column can't have a default value in OceanBase. builder.setDefaultValue(null); @@ -237,19 +271,46 @@ public OceanBaseColumn.Builder visit(DateType dateType) { return builder; } + @Override + public OceanBaseColumn.Builder visit(TimeType timeType) { + builder.setDataType(TIME); + builder.setNullable(timeType.isNullable()); + builder.setColumnSize( + Math.min(timeType.getPrecision(), TimestampType.DEFAULT_PRECISION)); + return builder; + } + + /** + * The maximum precision of the TIMESTAMP and DATETIME type is 6 in OceanBase. 1. + * TimestampType with timezone mapping to type TIMESTAMP in oceanbase 2. TimestampType + * without timezone mapping to type DATETIME in oceanbase + */ @Override public OceanBaseColumn.Builder visit(TimestampType timestampType) { - builder.setDataType(TIMESTAMP); + builder.setDataType(DATETIME); builder.setNullable(timestampType.isNullable()); builder.setColumnSize( Math.min(timestampType.getPrecision(), TimestampType.DEFAULT_PRECISION)); return builder; } + @Override + public OceanBaseColumn.Builder visit(ZonedTimestampType zonedTimestampType) { + builder.setDataType(TIMESTAMP); + builder.setNullable(zonedTimestampType.isNullable()); + builder.setColumnSize( + Math.min(zonedTimestampType.getPrecision(), TimestampType.DEFAULT_PRECISION)); + return builder; + } + @Override public OceanBaseColumn.Builder visit(LocalZonedTimestampType localZonedTimestampType) { - builder.setDataType(DATETIME); + builder.setDataType(TIMESTAMP); builder.setNullable(localZonedTimestampType.isNullable()); + builder.setColumnSize( + Math.min( + localZonedTimestampType.getPrecision(), + TimestampType.DEFAULT_PRECISION)); return builder; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java index 42121778b14..1582abae602 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseMetadataApplierTest.java @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.BooleanType; import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.common.types.IntType; import org.apache.flink.cdc.common.types.LocalZonedTimestampType; import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils; @@ -126,9 +127,214 @@ public void testCreateTable() { new OceanBaseColumn.Builder() .setColumnName("col3") .setOrdinalPosition(2) + .setDataType("timestamp") + .setNullable(true) + .build()); + OceanBaseTable expectTable = + new OceanBaseTable.Builder() + .setDatabaseName(tableId.getSchemaName()) + .setTableName(tableId.getTableName()) + .setTableType(OceanBaseTable.TableType.PRIMARY_KEY) + .setColumns(columns) + .setTableKeys(schema.primaryKeys()) + .build(); + + assertEquals(expectTable, actualTable); + } + + @Test + public void testCreateTableWithAllType() { + TableId tableId = TableId.parse("test.tbl6"); + Schema schema = + Schema.newBuilder() + .physicalColumn("col1", new IntType(false)) + .physicalColumn("col2", DataTypes.BOOLEAN()) + .physicalColumn("col3", DataTypes.TIMESTAMP_LTZ()) + .physicalColumn("col4", DataTypes.BYTES()) + .physicalColumn("col5", DataTypes.TINYINT()) + .physicalColumn("col6", DataTypes.SMALLINT()) + .physicalColumn("col7", DataTypes.BIGINT()) + .physicalColumn("col8", DataTypes.FLOAT()) + .physicalColumn("col9", DataTypes.DOUBLE()) + .physicalColumn("col10", DataTypes.DECIMAL(6, 3)) + .physicalColumn("col11", DataTypes.CHAR(5)) + .physicalColumn("col12", DataTypes.VARCHAR(10)) + .physicalColumn("col13", DataTypes.STRING()) + .physicalColumn("col14", DataTypes.DATE()) + .physicalColumn("col15", DataTypes.TIME()) + .physicalColumn("col16", DataTypes.TIME(6)) + .physicalColumn("col17", DataTypes.TIMESTAMP()) + .physicalColumn("col18", DataTypes.TIMESTAMP(3)) + .physicalColumn("col19", DataTypes.TIMESTAMP_LTZ(3)) + .physicalColumn("col20", DataTypes.TIMESTAMP_TZ()) + .physicalColumn("col21", DataTypes.TIMESTAMP_TZ(3)) + .primaryKey("col1") + .build(); + CreateTableEvent createTableEvent = new CreateTableEvent(tableId, schema); + metadataApplier.applySchemaChange(createTableEvent); + + OceanBaseTable actualTable = + catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); + assertNotNull(actualTable); + + List columns = new ArrayList<>(); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col1") + .setOrdinalPosition(0) + .setDataType("int") + .setNumericScale(0) + .setNullable(false) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col2") + .setOrdinalPosition(1) + .setDataType("tinyint") + .setNumericScale(0) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col3") + .setOrdinalPosition(2) + .setDataType("timestamp") + .setNullable(true) + .build()); + + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col4") + .setOrdinalPosition(3) + .setDataType("longblob") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col5") + .setOrdinalPosition(4) + .setDataType("tinyint") + .setNumericScale(0) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col6") + .setOrdinalPosition(5) + .setDataType("smallint") + .setNumericScale(0) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col7") + .setOrdinalPosition(6) + .setDataType("bigint") + .setNumericScale(0) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col8") + .setOrdinalPosition(7) + .setDataType("float") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col9") + .setOrdinalPosition(8) + .setDataType("double") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col10") + .setOrdinalPosition(9) + .setDataType("decimal") + .setNumericScale(3) + .setColumnSize(6) + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col11") + .setOrdinalPosition(10) + .setDataType("char") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col12") + .setOrdinalPosition(11) + .setDataType("varchar") + .setNullable(true) + .build()); + + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col13") + .setOrdinalPosition(12) + .setDataType("text") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col14") + .setOrdinalPosition(13) + .setDataType("date") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col15") + .setOrdinalPosition(14) + .setDataType("time") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col16") + .setOrdinalPosition(15) + .setDataType("time") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col17") + .setOrdinalPosition(16) + .setDataType("datetime") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col18") + .setOrdinalPosition(17) .setDataType("datetime") .setNullable(true) .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col19") + .setOrdinalPosition(18) + .setDataType("timestamp") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col20") + .setOrdinalPosition(19) + .setDataType("timestamp") + .setNullable(true) + .build()); + columns.add( + new OceanBaseColumn.Builder() + .setColumnName("col21") + .setOrdinalPosition(20) + .setDataType("timestamp") + .setNullable(true) + .build()); + OceanBaseTable expectTable = new OceanBaseTable.Builder() .setDatabaseName(tableId.getSchemaName()) @@ -138,8 +344,6 @@ public void testCreateTable() { .setTableKeys(schema.primaryKeys()) .build(); - System.out.println(expectTable); - System.out.println(actualTable); assertEquals(expectTable, actualTable); } @@ -247,8 +451,6 @@ public void testAlterColumnType() { OceanBaseTable changeTable = catalog.getTable(tableId.getSchemaName(), tableId.getTableName()).orElse(null); - System.out.println(actualTable); - System.out.println(changeTable); Assert.assertNotEquals( actualTable.getColumn("col2").getDataType(), changeTable.getColumn("col2").getDataType()); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java index 3abf1923dfa..a3f3d46d825 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseTestMySQLCatalog.java @@ -50,7 +50,7 @@ public Optional getTable(String databaseName, String tableName) "table name cannot be null or empty."); final String tableSchemaQuery = - "SELECT `COLUMN_NAME`, `DATA_TYPE`, `ORDINAL_POSITION`, `NUMERIC_SCALE`, " + "SELECT `COLUMN_NAME`, `DATA_TYPE`, `ORDINAL_POSITION`, `NUMERIC_SCALE`, `NUMERIC_PRECISION`, " + "`IS_NULLABLE`, `COLUMN_KEY`, `COLUMN_COMMENT` FROM `information_schema`.`COLUMNS` " + "WHERE `TABLE_SCHEMA`=? AND `TABLE_NAME`=?;"; @@ -70,6 +70,10 @@ public Optional getTable(String databaseName, String tableName) if (resultSet.wasNull()) { scale = null; } + Integer precision = null; + if ("decimal".equalsIgnoreCase(resultSet.getString("DATA_TYPE"))) { + precision = resultSet.getInt("NUMERIC_PRECISION"); + } String isNullable = resultSet.getString("IS_NULLABLE"); String comment = resultSet.getString("COLUMN_COMMENT"); OceanBaseColumn column = @@ -78,6 +82,7 @@ public Optional getTable(String databaseName, String tableName) .setOrdinalPosition(position - 1) .setDataType(type) .setNumericScale(scale) + .setColumnSize(precision) .setNullable( isNullable == null || !isNullable.equalsIgnoreCase("NO")) diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index dbeb50dfee4..42c85bf510a 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -166,10 +166,10 @@ public void testSyncWholeDatabase() throws Exception { customerList, Matchers.containsInAnyOrder( Stream.of( - "101,user_1,Shanghai,123567891234,2023-12-12 11:00:11.0", - "102,user_2,Shanghai,123567891234,2023-12-12 11:00:11.0", - "103,user_3,Shanghai,123567891234,2023-12-12 11:00:11.0", - "104,user_4,Shanghai,123567891234,2023-12-12 11:00:11.0") + "101,user_1,Shanghai,123567891234,2023-12-12T11:00:11", + "102,user_2,Shanghai,123567891234,2023-12-12T11:00:11", + "103,user_3,Shanghai,123567891234,2023-12-12T11:00:11", + "104,user_4,Shanghai,123567891234,2023-12-12T11:00:11") .toArray())); // generate binlogs