From 521a211c0d7441000e4f26f5fa1b49c13decf7cb Mon Sep 17 00:00:00 2001 From: He Wang Date: Mon, 11 Mar 2024 21:24:31 +0800 Subject: [PATCH] [FLINK-34639]Support DebeziumDeserializationSchema in OceanBase source connector --- .../flink-connector-oceanbase-cdc/pom.xml | 13 +- .../connectors/oceanbase/OceanBaseSource.java | 151 +++-- .../oceanbase/source/OceanBaseConnection.java | 166 ++++- ...anBaseDeserializationConverterFactory.java | 74 ++ .../source/OceanBaseRichSourceFunction.java | 287 +++++--- .../oceanbase/source/OceanBaseSchema.java | 80 +++ .../source/OceanBaseValueConverters.java | 504 ++++++++++++++ ...RowDataOceanBaseDeserializationSchema.java | 632 ------------------ .../config/OceanBaseConnectorConfig.java | 98 +++ .../schema/OceanBaseDatabaseSchema.java | 50 ++ .../source/schema/OceanBaseSourceInfo.java | 94 +++ .../OceanBaseSourceInfoStructMaker.java | 72 ++ .../OceanBaseAppendMetadataCollector.java | 57 -- .../table/OceanBaseDeserializationSchema.java | 37 - .../table/OceanBaseMetadataConverter.java | 29 - .../table/OceanBaseReadableMetadata.java | 66 +- .../oceanbase/table/OceanBaseRecord.java | 133 ---- .../oceanbase/table/OceanBaseTableSource.java | 50 +- .../table/OceanBaseTableSourceFactory.java | 107 +-- .../oceanbase/table/StartupMode.java | 55 -- .../OceanBaseUtils.java} | 16 +- .../oceanbase/OceanBaseTestBase.java | 221 +++--- ...ase.java => OceanBaseMySQLModeITCase.java} | 384 +++++------ .../table/OceanBaseOracleModeITCase.java | 249 +++++++ .../table/OceanBaseTableFactoryTest.java | 22 +- .../src/test/resources/ddl/inventory_meta.sql | 42 -- .../ddl/{ => mysql}/column_type_test.sql | 3 +- .../test/resources/ddl/mysql/docker_init.sql | 17 + .../resources/ddl/{ => mysql}/inventory.sql | 4 +- .../resources/ddl/oracle/column_type_test.sql | 70 ++ 30 files changed, 2209 insertions(+), 1574 deletions(-) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/{source/OceanBaseDeserializationRuntimeConverter.java => utils/OceanBaseUtils.java} (69%) rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/{OceanBaseConnectorITCase.java => OceanBaseMySQLModeITCase.java} (62%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/{ => mysql}/column_type_test.sql (97%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql rename flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/{ => mysql}/inventory.sql (95%) create mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml index f755eb21b5b..6a0f777cbc5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml @@ -34,6 +34,13 @@ limitations under the License. jar + + + com.ververica + flink-cdc-base + ${project.version} + + com.ververica @@ -47,12 +54,6 @@ limitations under the License. - - com.ververica - flink-cdc-common - ${project.version} - - com.oceanbase diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java index 2af7536c968..4a27c5fabec 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java @@ -18,9 +18,9 @@ package org.apache.flink.cdc.connectors.oceanbase; import org.apache.flink.cdc.common.annotation.PublicEvolving; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseRichSourceFunction; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema; -import org.apache.flink.cdc.connectors.oceanbase.table.StartupMode; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; import org.apache.flink.streaming.api.functions.source.SourceFunction; import com.oceanbase.clogproxy.client.config.ClientConf; @@ -47,7 +47,7 @@ public static Builder builder() { public static class Builder { // common config - private StartupMode startupMode; + private StartupOptions startupOptions; private String username; private String password; private String tenantName; @@ -73,11 +73,12 @@ public static class Builder { private String configUrl; private String workingMode; private Properties obcdcProperties; + private Properties debeziumProperties; - private OceanBaseDeserializationSchema deserializer; + private DebeziumDeserializationSchema deserializer; - public Builder startupMode(StartupMode startupMode) { - this.startupMode = startupMode; + public Builder startupOptions(StartupOptions startupOptions) { + this.startupOptions = startupOptions; return this; } @@ -151,7 +152,7 @@ public Builder logProxyHost(String logProxyHost) { return this; } - public Builder logProxyPort(int logProxyPort) { + public Builder logProxyPort(Integer logProxyPort) { this.logProxyPort = logProxyPort; return this; } @@ -186,23 +187,44 @@ public Builder obcdcProperties(Properties obcdcProperties) { return this; } - public Builder deserializer(OceanBaseDeserializationSchema deserializer) { + public Builder debeziumProperties(Properties debeziumProperties) { + this.debeziumProperties = debeziumProperties; + return this; + } + + public Builder deserializer(DebeziumDeserializationSchema deserializer) { this.deserializer = deserializer; return this; } public SourceFunction build() { - switch (startupMode) { - case INITIAL: - checkNotNull(hostname, "hostname shouldn't be null on startup mode 'initial'"); - checkNotNull(port, "port shouldn't be null on startup mode 'initial'"); - checkNotNull( - compatibleMode, - "compatibleMode shouldn't be null on startup mode 'initial'"); - checkNotNull( - jdbcDriver, "jdbcDriver shouldn't be null on startup mode 'initial'"); - startupTimestamp = 0L; + checkNotNull(username, "username shouldn't be null"); + checkNotNull(password, "password shouldn't be null"); + checkNotNull(hostname, "hostname shouldn't be null"); + checkNotNull(port, "port shouldn't be null"); + + if (startupOptions == null) { + startupOptions = StartupOptions.initial(); + } + if (compatibleMode == null) { + compatibleMode = "mysql"; + } + if (jdbcDriver == null) { + jdbcDriver = "com.mysql.jdbc.Driver"; + } + + if (connectTimeout == null) { + connectTimeout = Duration.ofSeconds(30); + } + + if (serverTimeZone == null) { + serverTimeZone = "+00:00"; + } + + switch (startupOptions.startupMode) { + case SNAPSHOT: break; + case INITIAL: case LATEST_OFFSET: startupTimestamp = 0L; break; @@ -213,15 +235,9 @@ public SourceFunction build() { break; default: throw new UnsupportedOperationException( - startupMode + " mode is not supported."); + startupOptions.startupMode + " mode is not supported."); } - if (!startupMode.equals(StartupMode.INITIAL) - && (StringUtils.isNotEmpty(databaseName) - || StringUtils.isNotEmpty(tableName))) { - throw new IllegalArgumentException( - "If startup mode is not 'INITIAL', 'database-name' and 'table-name' must not be configured"); - } if (StringUtils.isNotEmpty(databaseName) || StringUtils.isNotEmpty(tableName)) { if (StringUtils.isEmpty(databaseName) || StringUtils.isEmpty(tableName)) { throw new IllegalArgumentException( @@ -233,57 +249,61 @@ public SourceFunction build() { "'database-name', 'table-name' or 'table-list' should be configured"); } - if (serverTimeZone == null) { - serverTimeZone = "+00:00"; - } + ClientConf clientConf = null; + ObReaderConfig obReaderConfig = null; - if (connectTimeout == null) { - connectTimeout = Duration.ofSeconds(30); - } + if (!startupOptions.isSnapshotOnly()) { - if (logProxyClientId == null) { - logProxyClientId = - String.format( - "%s_%s_%s", - ClientIdGenerator.generate(), - Thread.currentThread().getId(), - checkNotNull(tenantName)); - } - ClientConf clientConf = - ClientConf.builder() - .clientId(logProxyClientId) - .connectTimeoutMs((int) connectTimeout.toMillis()) - .build(); - - ObReaderConfig obReaderConfig = new ObReaderConfig(); - if (StringUtils.isNotEmpty(rsList)) { - obReaderConfig.setRsList(rsList); - } - if (StringUtils.isNotEmpty(configUrl)) { - obReaderConfig.setClusterUrl(configUrl); - } - if (StringUtils.isNotEmpty(workingMode)) { - obReaderConfig.setWorkingMode(workingMode); - } - obReaderConfig.setUsername(username); - obReaderConfig.setPassword(password); - obReaderConfig.setStartTimestamp(startupTimestamp); - obReaderConfig.setTimezone(serverTimeZone); - - if (obcdcProperties != null && !obcdcProperties.isEmpty()) { - Map extraConfigs = new HashMap<>(); - obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString())); - obReaderConfig.setExtraConfigs(extraConfigs); + checkNotNull(logProxyHost); + checkNotNull(logProxyPort); + checkNotNull(tenantName); + + if (logProxyClientId == null) { + logProxyClientId = + String.format( + "%s_%s_%s", + ClientIdGenerator.generate(), + Thread.currentThread().getId(), + tenantName); + } + clientConf = + ClientConf.builder() + .clientId(logProxyClientId) + .connectTimeoutMs((int) connectTimeout.toMillis()) + .build(); + + obReaderConfig = new ObReaderConfig(); + if (StringUtils.isNotEmpty(rsList)) { + obReaderConfig.setRsList(rsList); + } + if (StringUtils.isNotEmpty(configUrl)) { + obReaderConfig.setClusterUrl(configUrl); + } + if (StringUtils.isNotEmpty(workingMode)) { + obReaderConfig.setWorkingMode(workingMode); + } + obReaderConfig.setTableWhiteList(tenantName + ".*.*"); + obReaderConfig.setUsername(username); + obReaderConfig.setPassword(password); + obReaderConfig.setStartTimestamp(startupTimestamp); + obReaderConfig.setTimezone(serverTimeZone); + + if (obcdcProperties != null && !obcdcProperties.isEmpty()) { + Map extraConfigs = new HashMap<>(); + obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString())); + obReaderConfig.setExtraConfigs(extraConfigs); + } } return new OceanBaseRichSourceFunction<>( - StartupMode.INITIAL.equals(startupMode), + startupOptions, username, password, tenantName, databaseName, tableName, tableList, + serverTimeZone, connectTimeout, hostname, port, @@ -294,6 +314,7 @@ public SourceFunction build() { logProxyPort, clientConf, obReaderConfig, + debeziumProperties, deserializer); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java index 89f538d621b..99994d12895 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseConnection.java @@ -17,10 +17,14 @@ package org.apache.flink.cdc.connectors.oceanbase.source; +import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils; import org.apache.flink.util.FlinkRuntimeException; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,11 +32,17 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; +import java.sql.Types; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -41,13 +51,19 @@ public class OceanBaseConnection extends JdbcConnection { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseConnection.class); - private static final String QUOTED_CHARACTER = "`"; private static final Properties DEFAULT_JDBC_PROPERTIES = initializeDefaultJdbcProperties(); private static final String MYSQL_URL_PATTERN = "jdbc:mysql://${hostname}:${port}/?connectTimeout=${connectTimeout}"; private static final String OB_URL_PATTERN = "jdbc:oceanbase://${hostname}:${port}/?connectTimeout=${connectTimeout}"; + private static final int TYPE_BINARY_FLOAT = 100; + private static final int TYPE_BINARY_DOUBLE = 101; + private static final int TYPE_TIMESTAMP_WITH_TIME_ZONE = -101; + private static final int TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE = -102; + private static final int TYPE_INTERVAL_YEAR_TO_MONTH = -103; + private static final int TYPE_INTERVAL_DAY_TO_SECOND = -104; + private final String compatibleMode; public OceanBaseConnection( @@ -62,9 +78,10 @@ public OceanBaseConnection( ClassLoader classLoader) { super( config(hostname, port, user, password, timeout), - factory(jdbcDriver, jdbcProperties, classLoader), - QUOTED_CHARACTER, - QUOTED_CHARACTER); + JdbcConnection.patternBasedFactory( + formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader), + getQuote(compatibleMode) + "", + getQuote(compatibleMode) + ""); this.compatibleMode = compatibleMode; } @@ -86,7 +103,7 @@ private static String formatJdbcUrl(String jdbcDriver, Properties jdbcProperties combinedProperties.putAll(jdbcProperties); } String urlPattern = - jdbcDriver.toLowerCase().contains("oceanbase") ? OB_URL_PATTERN : MYSQL_URL_PATTERN; + OceanBaseUtils.isOceanBaseDriver(jdbcDriver) ? OB_URL_PATTERN : MYSQL_URL_PATTERN; StringBuilder jdbcUrlStringBuilder = new StringBuilder(urlPattern); combinedProperties.forEach( (key, value) -> { @@ -106,10 +123,8 @@ private static Properties initializeDefaultJdbcProperties() { return defaultJdbcProperties; } - private static JdbcConnection.ConnectionFactory factory( - String jdbcDriver, Properties jdbcProperties, ClassLoader classLoader) { - return JdbcConnection.patternBasedFactory( - formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader); + private static char getQuote(String compatibleMode) { + return "mysql".equalsIgnoreCase(compatibleMode) ? '`' : '"'; } /** @@ -155,11 +170,11 @@ public Optional getCurrentTimestamp() throws SQLException { * * @param dbPattern Database name pattern. * @param tbPattern Table name pattern. - * @return Table list. + * @return TableId list. * @throws SQLException If a database access error occurs. */ - public List getTables(String dbPattern, String tbPattern) throws SQLException { - List result = new ArrayList<>(); + public List getTables(String dbPattern, String tbPattern) throws SQLException { + List result = new ArrayList<>(); DatabaseMetaData metaData = connection().getMetaData(); switch (compatibleMode.toLowerCase()) { case "mysql": @@ -171,27 +186,28 @@ public List getTables(String dbPattern, String tbPattern) throws SQLExce for (String dbName : dbNames) { List tableNames = getResultList( - metaData.getTables(dbName, null, null, new String[] {"TABLE"}), + metaData.getTables(dbName, null, null, supportedTableTypes()), "TABLE_NAME"); tableNames.stream() .filter(tbName -> Pattern.matches(tbPattern, tbName)) - .forEach(tbName -> result.add(dbName + "." + tbName)); + .forEach(tbName -> result.add(new TableId(dbName, null, tbName))); } break; case "oracle": - dbNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM"); - dbNames = - dbNames.stream() - .filter(dbName -> Pattern.matches(dbPattern, dbName)) + List schemaNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM"); + schemaNames = + schemaNames.stream() + .filter(schemaName -> Pattern.matches(dbPattern, schemaName)) .collect(Collectors.toList()); - for (String dbName : dbNames) { + for (String schemaName : schemaNames) { List tableNames = getResultList( - metaData.getTables(null, dbName, null, new String[] {"TABLE"}), + metaData.getTables( + null, schemaName, null, supportedTableTypes()), "TABLE_NAME"); tableNames.stream() .filter(tbName -> Pattern.matches(tbPattern, tbName)) - .forEach(tbName -> result.add(dbName + "." + tbName)); + .forEach(tbName -> result.add(new TableId(null, schemaName, tbName))); } break; default: @@ -207,4 +223,112 @@ private List getResultList(ResultSet resultSet, String columnName) throw } return result; } + + @Override + protected String[] supportedTableTypes() { + return new String[] {"TABLE"}; + } + + @Override + public String quotedTableIdString(TableId tableId) { + return tableId.toQuotedString(getQuote(compatibleMode)); + } + + public void readSchemaForCapturedTables( + Tables tables, + String databaseCatalog, + String schemaNamePattern, + Tables.ColumnNameFilter columnFilter, + boolean removeTablesNotFoundInJdbc, + Set capturedTables) + throws SQLException { + + Set tableIdsBefore = new HashSet<>(tables.tableIds()); + + DatabaseMetaData metadata = connection().getMetaData(); + Map> columnsByTable = new HashMap<>(); + + for (TableId tableId : capturedTables) { + try (ResultSet columnMetadata = + metadata.getColumns( + databaseCatalog, schemaNamePattern, tableId.table(), null)) { + while (columnMetadata.next()) { + // add all whitelisted columns + readTableColumn(columnMetadata, tableId, columnFilter) + .ifPresent( + column -> { + columnsByTable + .computeIfAbsent(tableId, t -> new ArrayList<>()) + .add(column.create()); + }); + } + } + } + + // Read the metadata for the primary keys ... + for (Map.Entry> tableEntry : columnsByTable.entrySet()) { + // First get the primary key information, which must be done for *each* table ... + List pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey()); + + // Then define the table ... + List columns = tableEntry.getValue(); + Collections.sort(columns); + tables.overwriteTable(tableEntry.getKey(), columns, pkColumnNames, null); + } + + if (removeTablesNotFoundInJdbc) { + // Remove any definitions for tables that were not found in the database metadata ... + tableIdsBefore.removeAll(columnsByTable.keySet()); + tableIdsBefore.forEach(tables::removeTable); + } + } + + @Override + protected int resolveNativeType(String typeName) { + String upperCaseTypeName = typeName.toUpperCase(); + if (upperCaseTypeName.startsWith("JSON")) { + return Types.VARCHAR; + } + if (upperCaseTypeName.startsWith("NCHAR")) { + return Types.NCHAR; + } + if (upperCaseTypeName.startsWith("NVARCHAR2")) { + return Types.NVARCHAR; + } + if (upperCaseTypeName.startsWith("TIMESTAMP")) { + if (upperCaseTypeName.contains("WITH TIME ZONE")) { + return TYPE_TIMESTAMP_WITH_TIME_ZONE; + } + if (upperCaseTypeName.contains("WITH LOCAL TIME ZONE")) { + return TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE; + } + return Types.TIMESTAMP; + } + if (upperCaseTypeName.startsWith("INTERVAL")) { + if (upperCaseTypeName.contains("TO MONTH")) { + return TYPE_INTERVAL_YEAR_TO_MONTH; + } + if (upperCaseTypeName.contains("TO SECOND")) { + return TYPE_INTERVAL_DAY_TO_SECOND; + } + } + return Column.UNSET_INT_VALUE; + } + + @Override + protected int resolveJdbcType(int metadataJdbcType, int nativeType) { + switch (metadataJdbcType) { + case TYPE_BINARY_FLOAT: + return Types.REAL; + case TYPE_BINARY_DOUBLE: + return Types.DOUBLE; + case TYPE_TIMESTAMP_WITH_TIME_ZONE: + case TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case TYPE_INTERVAL_YEAR_TO_MONTH: + case TYPE_INTERVAL_DAY_TO_SECOND: + return Types.OTHER; + default: + return nativeType == Column.UNSET_INT_VALUE ? metadataJdbcType : nativeType; + } + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java new file mode 100644 index 00000000000..4bc8f6e2338 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationConverterFactory.java @@ -0,0 +1,74 @@ +/* + * 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.source; + +import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverter; +import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverterFactory; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.LogicalType; + +import org.apache.kafka.connect.data.Schema; + +import java.time.ZoneId; +import java.util.Optional; + +/** Used to create {@link DeserializationRuntimeConverterFactory} specified to OceanBase. */ +public class OceanBaseDeserializationConverterFactory { + + public static DeserializationRuntimeConverterFactory instance() { + return new DeserializationRuntimeConverterFactory() { + + private static final long serialVersionUID = 1L; + + @Override + public Optional createUserDefinedConverter( + LogicalType logicalType, ZoneId serverTimeZone) { + switch (logicalType.getTypeRoot()) { + case ARRAY: + return createArrayConverter(); + default: + return Optional.empty(); + } + } + }; + } + + private static Optional createArrayConverter() { + return Optional.of( + new DeserializationRuntimeConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(Object dbzObj, Schema schema) throws Exception { + if (dbzObj instanceof String) { + String[] enums = ((String) dbzObj).split(","); + StringData[] elements = new StringData[enums.length]; + for (int i = 0; i < enums.length; i++) { + elements[i] = StringData.fromString(enums[i]); + } + return new GenericArrayData(elements); + } + throw new IllegalArgumentException( + String.format( + "Unable convert to Flink ARRAY type from unexpected value '%s'", + dbzObj)); + } + }); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java index 92df4515fcc..5752d7d2691 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java @@ -23,8 +23,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseRecord; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseDatabaseSchema; +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSourceInfo; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -38,14 +41,23 @@ import com.oceanbase.clogproxy.client.config.ObReaderConfig; import com.oceanbase.clogproxy.client.exception.LogProxyClientException; import com.oceanbase.clogproxy.client.listener.RecordListener; +import com.oceanbase.oms.logmessage.DataMessage; import com.oceanbase.oms.logmessage.LogMessage; +import io.debezium.connector.SnapshotRecord; +import io.debezium.relational.TableId; +import io.debezium.relational.TableSchema; +import io.debezium.relational.history.TableChanges; import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.time.Duration; +import java.time.Instant; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; @@ -72,13 +84,14 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction private static final Logger LOG = LoggerFactory.getLogger(OceanBaseRichSourceFunction.class); - private final boolean snapshot; + private final StartupOptions startupOptions; private final String username; private final String password; private final String tenantName; private final String databaseName; private final String tableName; private final String tableList; + private final String serverTimeZone; private final Duration connectTimeout; private final String hostname; private final Integer port; @@ -86,14 +99,19 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction private final String jdbcDriver; private final Properties jdbcProperties; private final String logProxyHost; - private final int logProxyPort; + private final Integer logProxyPort; private final ClientConf logProxyClientConf; private final ObReaderConfig obReaderConfig; - private final OceanBaseDeserializationSchema deserializer; + private final Properties debeziumProperties; + private final DebeziumDeserializationSchema deserializer; - private final List changeRecordBuffer = new LinkedList<>(); + private final List changeRecordBuffer = new LinkedList<>(); - private transient Set tableSet; + private transient OceanBaseConnectorConfig connectorConfig; + private transient OceanBaseSourceInfo sourceInfo; + private transient Set tableSet; + private transient OceanBaseSchema obSchema; + private transient OceanBaseDatabaseSchema databaseSchema; private transient volatile long resolvedTimestamp; private transient volatile OceanBaseConnection snapshotConnection; private transient LogProxyClient logProxyClient; @@ -101,13 +119,14 @@ public class OceanBaseRichSourceFunction extends RichSourceFunction private transient OutputCollector outputCollector; public OceanBaseRichSourceFunction( - boolean snapshot, + StartupOptions startupOptions, String username, String password, String tenantName, String databaseName, String tableName, String tableList, + String serverTimeZone, Duration connectTimeout, String hostname, Integer port, @@ -115,27 +134,30 @@ public OceanBaseRichSourceFunction( String jdbcDriver, Properties jdbcProperties, String logProxyHost, - int logProxyPort, + Integer logProxyPort, ClientConf logProxyClientConf, ObReaderConfig obReaderConfig, - OceanBaseDeserializationSchema deserializer) { - this.snapshot = checkNotNull(snapshot); + Properties debeziumProperties, + DebeziumDeserializationSchema deserializer) { + this.startupOptions = checkNotNull(startupOptions); this.username = checkNotNull(username); this.password = checkNotNull(password); - this.tenantName = checkNotNull(tenantName); + this.tenantName = tenantName; this.databaseName = databaseName; this.tableName = tableName; this.tableList = tableList; + this.serverTimeZone = checkNotNull(serverTimeZone); this.connectTimeout = checkNotNull(connectTimeout); - this.hostname = hostname; - this.port = port; - this.compatibleMode = compatibleMode; - this.jdbcDriver = jdbcDriver; + this.hostname = checkNotNull(hostname); + this.port = checkNotNull(port); + this.compatibleMode = checkNotNull(compatibleMode); + this.jdbcDriver = checkNotNull(jdbcDriver); this.jdbcProperties = jdbcProperties; - this.logProxyHost = checkNotNull(logProxyHost); - this.logProxyPort = checkNotNull(logProxyPort); - this.logProxyClientConf = checkNotNull(logProxyClientConf); - this.obReaderConfig = checkNotNull(obReaderConfig); + this.logProxyHost = logProxyHost; + this.logProxyPort = logProxyPort; + this.logProxyClientConf = logProxyClientConf; + this.obReaderConfig = obReaderConfig; + this.debeziumProperties = debeziumProperties; this.deserializer = checkNotNull(deserializer); } @@ -143,6 +165,10 @@ public OceanBaseRichSourceFunction( public void open(final Configuration config) throws Exception { super.open(config); this.outputCollector = new OutputCollector<>(); + this.connectorConfig = + new OceanBaseConnectorConfig( + compatibleMode, serverTimeZone, tenantName, debeziumProperties); + this.sourceInfo = new OceanBaseSourceInfo(connectorConfig); } @Override @@ -152,27 +178,28 @@ public void run(SourceContext ctx) throws Exception { LOG.info("Start to initial table whitelist"); initTableWhiteList(); - if (shouldReadSnapshot()) { + if (resolvedTimestamp <= 0 && !startupOptions.isStreamOnly()) { + sourceInfo.setSnapshot(SnapshotRecord.TRUE); long startTimestamp = getSnapshotConnection().getCurrentTimestampS(); LOG.info("Snapshot reading started from timestamp: {}", startTimestamp); readSnapshotRecords(); + sourceInfo.setSnapshot(SnapshotRecord.FALSE); LOG.info("Snapshot reading finished"); resolvedTimestamp = startTimestamp; } else { LOG.info("Snapshot reading skipped"); } - LOG.info("Change events reading started"); - readChangeRecords(); + if (!startupOptions.isSnapshotOnly()) { + sourceInfo.setSnapshot(SnapshotRecord.FALSE); + LOG.info("Change events reading started"); + readChangeRecords(); + } } finally { cancel(); } } - private boolean shouldReadSnapshot() { - return resolvedTimestamp <= 0 && snapshot; - } - private OceanBaseConnection getSnapshotConnection() { if (snapshotConnection == null) { snapshotConnection = @@ -201,27 +228,35 @@ private void closeSnapshotConnection() { } } + private TableId tableId(String databaseName, String tableName) { + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return new TableId(databaseName, null, tableName); + } + return new TableId(null, databaseName, tableName); + } + private void initTableWhiteList() { if (tableSet != null && !tableSet.isEmpty()) { return; } - final Set localTableSet = new HashSet<>(); + final Set localTableSet = new HashSet<>(); if (StringUtils.isNotBlank(tableList)) { for (String s : tableList.split(",")) { if (StringUtils.isNotBlank(s)) { - String[] schema = s.split("\\."); - localTableSet.add(String.format("%s.%s", schema[0].trim(), schema[1].trim())); + String[] arr = s.split("\\."); + TableId tableId = tableId(arr[0].trim(), arr[1].trim()); + localTableSet.add(tableId); } } } if (StringUtils.isNotBlank(databaseName) && StringUtils.isNotBlank(tableName)) { try { - List tables = getSnapshotConnection().getTables(databaseName, tableName); - LOG.info("Pattern matched tables: {}", tables); - localTableSet.addAll(tables); + List tableIds = getSnapshotConnection().getTables(databaseName, tableName); + LOG.info("Pattern matched tables: {}", tableIds); + localTableSet.addAll(tableIds); } catch (SQLException e) { LOG.error( String.format( @@ -238,50 +273,73 @@ private void initTableWhiteList() { LOG.info("Table list: {}", localTableSet); this.tableSet = localTableSet; - this.obReaderConfig.setTableWhiteList(String.format("%s.*.*", tenantName)); + } + + private TableSchema getTableSchema(TableId tableId) { + if (databaseSchema == null) { + databaseSchema = new OceanBaseDatabaseSchema(connectorConfig, false); + } + TableSchema tableSchema = databaseSchema.schemaFor(tableId); + if (tableSchema != null) { + return tableSchema; + } + + if (obSchema == null) { + obSchema = new OceanBaseSchema(); + } + TableChanges.TableChange tableChange = + obSchema.getTableSchema(getSnapshotConnection(), tableId); + databaseSchema.refresh(tableChange.getTable()); + return databaseSchema.schemaFor(tableId); } protected void readSnapshotRecords() { - tableSet.forEach( - table -> { - String[] schema = table.split("\\."); - readSnapshotRecordsByTable(schema[0], schema[1]); - }); + tableSet.forEach(this::readSnapshotRecordsByTable); } - private void readSnapshotRecordsByTable(String databaseName, String tableName) { - OceanBaseRecord.SourceInfo sourceInfo = - new OceanBaseRecord.SourceInfo( - tenantName, databaseName, tableName, resolvedTimestamp); - String fullName; - if ("mysql".equalsIgnoreCase(compatibleMode)) { - fullName = String.format("`%s`.`%s`", databaseName, tableName); - } else { - fullName = String.format("%s.%s", databaseName, tableName); - } - try { - LOG.info("Start to read snapshot from {}", fullName); - getSnapshotConnection() - .query( - "SELECT * FROM " + fullName, - rs -> { - ResultSetMetaData metaData = rs.getMetaData(); - while (rs.next()) { - Map fieldMap = new HashMap<>(); - for (int i = 0; i < metaData.getColumnCount(); i++) { - fieldMap.put( - metaData.getColumnName(i + 1), rs.getObject(i + 1)); - } - OceanBaseRecord record = - new OceanBaseRecord(sourceInfo, fieldMap); - try { - deserializer.deserialize(record, outputCollector); - } catch (Exception e) { - LOG.error("Deserialize snapshot record failed ", e); - throw new FlinkRuntimeException(e); - } - } - }); + private void readSnapshotRecordsByTable(TableId tableId) { + String fullName = getSnapshotConnection().quotedTableIdString(tableId); + sourceInfo.tableEvent(tableId); + try (OceanBaseConnection connection = getSnapshotConnection()) { + LOG.info("Start to read snapshot from {}", connection.quotedTableIdString(tableId)); + connection.query( + "SELECT * FROM " + fullName, + rs -> { + TableSchema tableSchema = getTableSchema(tableId); + List fields = tableSchema.valueSchema().fields(); + + while (rs.next()) { + Object[] fieldValues = new Object[fields.size()]; + for (Field field : fields) { + fieldValues[field.index()] = rs.getObject(field.name()); + } + Struct value = tableSchema.valueFromColumnData(fieldValues); + + Instant now = Instant.now(); + sourceInfo.setSourceTime(now); + + Struct struct = + tableSchema + .getEnvelopeSchema() + .read(value, sourceInfo.struct(), now); + try { + deserializer.deserialize( + new SourceRecord( + null, + null, + tableId.identifier(), + null, + null, + null, + struct.schema(), + struct), + outputCollector); + } catch (Exception e) { + LOG.error("Deserialize snapshot record failed ", e); + throw new FlinkRuntimeException(e); + } + } + }); LOG.info("Read snapshot from {} finished", fullName); } catch (SQLException e) { LOG.error("Read snapshot from table " + fullName + " failed", e); @@ -321,7 +379,7 @@ public void notify(LogMessage message) { if (!started) { break; } - OceanBaseRecord record = getChangeRecord(message); + SourceRecord record = getChangeRecord(message); if (record != null) { changeRecordBuffer.add(record); } @@ -375,18 +433,81 @@ public void onException(LogProxyClientException e) { logProxyClient.join(); } - private OceanBaseRecord getChangeRecord(LogMessage message) { + private SourceRecord getChangeRecord(LogMessage message) { String databaseName = message.getDbName().replace(tenantName + ".", ""); - if (!tableSet.contains(String.format("%s.%s", databaseName, message.getTableName()))) { + TableId tableId = tableId(databaseName, message.getTableName()); + if (!tableSet.contains(tableId)) { + return null; + } + + sourceInfo.tableEvent(tableId); + sourceInfo.setSourceTime(Instant.now()); + Struct source = sourceInfo.struct(); + + TableSchema tableSchema = getTableSchema(tableId); + Struct struct; + Schema valueSchema = tableSchema.valueSchema(); + List fields = valueSchema.fields(); + Struct before, after; + Object[] beforeFieldValues, afterFieldValues; + Map beforeValueMap = new HashMap<>(); + Map afterValueMap = new HashMap<>(); + message.getFieldList() + .forEach( + field -> { + if (field.isPrev()) { + beforeValueMap.put(field.getFieldname(), getFieldValue(field)); + } else { + afterValueMap.put(field.getFieldname(), getFieldValue(field)); + } + }); + switch (message.getOpt()) { + case INSERT: + afterFieldValues = new Object[fields.size()]; + for (Field field : fields) { + afterFieldValues[field.index()] = afterValueMap.get(field.name()); + } + after = tableSchema.valueFromColumnData(afterFieldValues); + struct = tableSchema.getEnvelopeSchema().create(after, source, Instant.now()); + break; + case DELETE: + beforeFieldValues = new Object[fields.size()]; + for (Field field : fields) { + beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); + } + before = tableSchema.valueFromColumnData(beforeFieldValues); + struct = tableSchema.getEnvelopeSchema().delete(before, source, Instant.now()); + break; + case UPDATE: + beforeFieldValues = new Object[fields.size()]; + afterFieldValues = new Object[fields.size()]; + for (Field field : fields) { + beforeFieldValues[field.index()] = beforeValueMap.get(field.name()); + afterFieldValues[field.index()] = afterValueMap.get(field.name()); + } + before = tableSchema.valueFromColumnData(beforeFieldValues); + after = tableSchema.valueFromColumnData(afterFieldValues); + struct = + tableSchema + .getEnvelopeSchema() + .update(before, after, source, Instant.now()); + break; + default: + throw new UnsupportedOperationException(); + } + return new SourceRecord( + null, null, tableId.identifier(), null, null, null, struct.schema(), struct); + } + + private Object getFieldValue(DataMessage.Record.Field field) { + if (field.getValue() == null) { return null; } - OceanBaseRecord.SourceInfo sourceInfo = - new OceanBaseRecord.SourceInfo( - tenantName, - databaseName, - message.getTableName(), - Long.parseLong(message.getSafeTimestamp())); - return new OceanBaseRecord(sourceInfo, message.getOpt(), message.getFieldList()); + String encoding = field.getEncoding(); + if ("binary".equalsIgnoreCase(encoding)) { + return field.getValue().getBytes(); + } + return field.getValue().toString(encoding); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java new file mode 100644 index 00000000000..0444b36efe7 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseSchema.java @@ -0,0 +1,80 @@ +/* + * 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.source; + +import org.apache.flink.util.FlinkRuntimeException; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.relational.history.TableChanges; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** A component used to get schema by table path. */ +public class OceanBaseSchema { + + private final Map schemasByTableId; + + public OceanBaseSchema() { + this.schemasByTableId = new HashMap<>(); + } + + public TableChanges.TableChange getTableSchema(JdbcConnection connection, TableId tableId) { + TableChanges.TableChange schema = schemasByTableId.get(tableId); + if (schema == null) { + schema = readTableSchema(connection, tableId); + schemasByTableId.put(tableId, schema); + } + return schema; + } + + private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { + OceanBaseConnection connection = (OceanBaseConnection) jdbc; + Set tableIdSet = new HashSet<>(); + tableIdSet.add(tableId); + + final Map tableChangeMap = new HashMap<>(); + Tables tables = new Tables(); + tables.overwriteTable(tables.editOrCreateTable(tableId).create()); + + try { + connection.readSchemaForCapturedTables( + tables, tableId.catalog(), tableId.schema(), null, false, tableIdSet); + Table table = tables.forTable(tableId); + TableChanges.TableChange tableChange = + new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table); + tableChangeMap.put(tableId, tableChange); + } catch (SQLException e) { + throw new FlinkRuntimeException( + String.format("Failed to read schema for table %s ", tableId), e); + } + + if (!tableChangeMap.containsKey(tableId)) { + throw new FlinkRuntimeException( + String.format("Can't obtain schema for table %s ", tableId)); + } + + return tableChangeMap.get(tableId); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java new file mode 100644 index 00000000000..a14722ca6b6 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseValueConverters.java @@ -0,0 +1,504 @@ +/* + * 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.source; + +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.data.Bits; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.jdbc.JdbcValueConverters; +import io.debezium.relational.Column; +import io.debezium.relational.ValueConverter; +import io.debezium.time.MicroTimestamp; +import io.debezium.time.NanoTimestamp; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.SchemaBuilder; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Date; +import java.sql.SQLException; +import java.sql.Time; +import java.sql.Types; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.Locale; + +/** JdbcValueConverters for OceanBase. */ +public class OceanBaseValueConverters extends JdbcValueConverters { + + public static final String EMPTY_BLOB_FUNCTION = "EMPTY_BLOB()"; + public static final String EMPTY_CLOB_FUNCTION = "EMPTY_CLOB()"; + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern("yyyy-MM-dd HH:mm:ss") + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .toFormatter(); + + private static final DateTimeFormatter TIMESTAMP_AM_PM_SHORT_FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern("dd-MMM-yy hh.mm.ss") + .optionalStart() + .appendPattern(".") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false) + .optionalEnd() + .appendPattern(" a") + .toFormatter(Locale.ENGLISH); + + private final String compatibleMode; + private final String serverTimezone; + + public OceanBaseValueConverters(OceanBaseConnectorConfig connectorConfig) { + super( + connectorConfig.getDecimalMode(), + connectorConfig.getTemporalPrecisionMode(), + ZoneOffset.UTC, + x -> x, + BigIntUnsignedMode.PRECISE, + connectorConfig.binaryHandlingMode()); + this.compatibleMode = connectorConfig.getCompatibleMode(); + this.serverTimezone = connectorConfig.getServerTimeZone(); + } + + @Override + protected int getTimePrecision(Column column) { + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return super.getTimePrecision(column); + } + return column.scale().orElse(0); + } + + protected boolean isUnsignedColumn(Column column) { + return column.typeName().toUpperCase().contains("UNSIGNED"); + } + + @Override + public SchemaBuilder schemaBuilder(Column column) { + logger.debug( + "Building schema for column {} of type {} named {} with constraints ({},{})", + column.name(), + column.jdbcType(), + column.typeName(), + column.length(), + column.scale()); + + switch (column.jdbcType()) { + case Types.BIT: + if (column.length() > 1) { + return Bits.builder(column.length()); + } + return SchemaBuilder.bool(); + case Types.TINYINT: + if (column.length() == 1) { + return SchemaBuilder.bool(); + } + if (isUnsignedColumn(column)) { + return SchemaBuilder.int16(); + } + return SchemaBuilder.int8(); + case Types.SMALLINT: + if (isUnsignedColumn(column)) { + return SchemaBuilder.int32(); + } + return SchemaBuilder.int16(); + case Types.INTEGER: + if (!column.typeName().toUpperCase().startsWith("MEDIUMINT") + && isUnsignedColumn(column)) { + return SchemaBuilder.int64(); + } + return SchemaBuilder.int32(); + case Types.BIGINT: + if (isUnsignedColumn(column)) { + return Decimal.builder(0); + } + return SchemaBuilder.int64(); + case Types.FLOAT: + return getDecimalSchema(column); + case Types.NUMERIC: + case Types.DECIMAL: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return getDecimalSchema(column); + } + return getNumericSchema(column); + case Types.REAL: + return SchemaBuilder.float32(); + case Types.DOUBLE: + return SchemaBuilder.float64(); + case Types.DATE: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + if (column.typeName().equalsIgnoreCase("YEAR")) { + return io.debezium.time.Year.builder(); + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + return io.debezium.time.Date.builder(); + } + return org.apache.kafka.connect.data.Date.builder(); + } + return getTimestampSchema(column); + case Types.TIME: + if (adaptiveTimeMicrosecondsPrecisionMode) { + return io.debezium.time.MicroTime.builder(); + } + if (adaptiveTimePrecisionMode) { + if (getTimePrecision(column) <= 3) { + return io.debezium.time.Time.builder(); + } + if (getTimePrecision(column) <= 6) { + return io.debezium.time.MicroTime.builder(); + } + return io.debezium.time.NanoTime.builder(); + } + return org.apache.kafka.connect.data.Time.builder(); + case Types.TIMESTAMP: + return getTimestampSchema(column); + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.CLOB: + return SchemaBuilder.string(); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + case Types.BLOB: + return binaryMode.getSchema(); + default: + return super.schemaBuilder(column); + } + } + + protected SchemaBuilder getNumericSchema(Column column) { + if (column.scale().isPresent()) { + int scale = column.scale().get(); + if (scale <= 0) { + int width = column.length() - scale; + if (width < 3) { + return SchemaBuilder.int8(); + } else if (width < 5) { + return SchemaBuilder.int16(); + } else if (width < 10) { + return SchemaBuilder.int32(); + } else if (width < 19) { + return SchemaBuilder.int64(); + } + } + } + return getDecimalSchema(column); + } + + protected SchemaBuilder getDecimalSchema(Column column) { + return SpecialValueDecimal.builder(decimalMode, column.length(), column.scale().orElse(0)); + } + + protected SchemaBuilder getTimestampSchema(Column column) { + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + if (getTimePrecision(column) <= 3) { + return io.debezium.time.Timestamp.builder(); + } + if (getTimePrecision(column) <= 6) { + return MicroTimestamp.builder(); + } + return NanoTimestamp.builder(); + } + return org.apache.kafka.connect.data.Timestamp.builder(); + } + + @Override + public ValueConverter converter(Column column, Field fieldDefn) { + switch (column.jdbcType()) { + case Types.BIT: + return convertBits(column, fieldDefn); + case Types.TINYINT: + if (column.length() == 1) { + return data -> convertBit(column, fieldDefn, data); + } + if (isUnsignedColumn(column)) { + return data -> convertSmallInt(column, fieldDefn, data); + } + return data -> convertTinyInt(column, fieldDefn, data); + case Types.SMALLINT: + if (isUnsignedColumn(column)) { + return data -> convertInteger(column, fieldDefn, data); + } + return data -> convertSmallInt(column, fieldDefn, data); + case Types.INTEGER: + if (column.typeName().toUpperCase().startsWith("MEDIUMINT")) { + return data -> convertInteger(column, fieldDefn, data); + } + if (isUnsignedColumn(column)) { + return data -> convertBigInt(column, fieldDefn, data); + } + return data -> convertInteger(column, fieldDefn, data); + case Types.BIGINT: + if (isUnsignedColumn(column)) { + switch (bigIntUnsignedMode) { + case LONG: + return (data) -> convertBigInt(column, fieldDefn, data); + case PRECISE: + return (data) -> convertUnsignedBigint(column, fieldDefn, data); + } + } + return (data) -> convertBigInt(column, fieldDefn, data); + case Types.FLOAT: + return data -> convertDecimal(column, fieldDefn, data); + case Types.NUMERIC: + case Types.DECIMAL: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + return data -> convertDecimal(column, fieldDefn, data); + } + return data -> convertNumeric(column, fieldDefn, data); + case Types.REAL: + return data -> convertReal(column, fieldDefn, data); + case Types.DOUBLE: + return data -> convertDouble(column, fieldDefn, data); + case Types.DATE: + if ("mysql".equalsIgnoreCase(compatibleMode)) { + if (column.typeName().equalsIgnoreCase("YEAR")) { + return (data) -> convertYearToInt(column, fieldDefn, data); + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + return (data) -> convertDateToEpochDays(column, fieldDefn, data); + } + return (data) -> convertDateToEpochDaysAsDate(column, fieldDefn, data); + } + return (data) -> convertTimestamp(column, fieldDefn, data); + case Types.TIME: + return (data) -> convertTime(column, fieldDefn, data); + case Types.TIMESTAMP: + return data -> convertTimestamp(column, fieldDefn, data); + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.CLOB: + return data -> convertString(column, fieldDefn, data); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + case Types.BLOB: + return (data) -> convertBinary(column, fieldDefn, data, binaryMode); + default: + return super.converter(column, fieldDefn); + } + } + + @Override + protected Object convertBits(Column column, Field fieldDefn, Object data, int numBytes) { + if (data instanceof String) { + return ByteBuffer.allocate(numBytes).putLong(Long.parseLong((String) data)).array(); + } + return super.convertBits(column, fieldDefn, data, numBytes); + } + + @Override + protected Object convertBit(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Boolean.parseBoolean((String) data) || "1".equals(data); + } + return super.convertBit(column, fieldDefn, data); + } + + @Override + protected Object convertTinyInt(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Byte.parseByte((String) data); + } + if (data instanceof Number) { + return ((Number) data).byteValue(); + } + throw new IllegalArgumentException( + "Unexpected value for JDBC type " + + column.jdbcType() + + " and column " + + column + + ": class=" + + data.getClass()); + } + + @Override + protected Object convertBigInt(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return new BigInteger((String) data).longValue(); + } + return super.convertBigInt(column, fieldDefn, data); + } + + protected Object convertUnsignedBigint(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return new BigDecimal((String) data); + } + if (data instanceof BigInteger) { + return new BigDecimal((BigInteger) data); + } + return convertDecimal(column, fieldDefn, data); + } + + @Override + protected Object convertReal(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Float.parseFloat((String) data); + } + return super.convertReal(column, fieldDefn, data); + } + + @Override + protected Object convertDouble(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + return Double.parseDouble((String) data); + } + return super.convertDouble(column, fieldDefn, data); + } + + @Override + protected Object convertNumeric(Column column, Field fieldDefn, Object data) { + if (column.scale().isPresent()) { + int scale = column.scale().get(); + + if (scale <= 0) { + int width = column.length() - scale; + if (width < 3) { + return convertTinyInt(column, fieldDefn, data); + } else if (width < 5) { + return convertSmallInt(column, fieldDefn, data); + } else if (width < 10) { + return convertInteger(column, fieldDefn, data); + } else if (width < 19) { + return convertBigInt(column, fieldDefn, data); + } + } + } + return convertDecimal(column, fieldDefn, data); + } + + protected Object convertYearToInt(Column column, Field fieldDefn, Object data) { + if (data instanceof Date) { + return ((Date) data).toLocalDate().getYear(); + } + return convertInteger(column, fieldDefn, data); + } + + @Override + protected Object convertDateToEpochDays(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Date.valueOf((String) data); + } + return super.convertDateToEpochDays(column, fieldDefn, data); + } + + @Override + protected Object convertDateToEpochDaysAsDate(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Date.valueOf((String) data); + } + return super.convertDateToEpochDaysAsDate(column, fieldDefn, data); + } + + @Override + protected Object convertTime(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = Time.valueOf((String) data); + } + return super.convertTime(column, fieldDefn, data); + } + + protected Object convertTimestamp(Column column, Field fieldDefn, Object data) { + if (data instanceof String) { + data = resolveTimestampStringAsInstant((String) data); + } + if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) { + if (getTimePrecision(column) <= 3) { + return convertTimestampToEpochMillis(column, fieldDefn, data); + } + if (getTimePrecision(column) <= 6) { + return convertTimestampToEpochMicros(column, fieldDefn, data); + } + return convertTimestampToEpochNanos(column, fieldDefn, data); + } + return convertTimestampToEpochMillisAsDate(column, fieldDefn, data); + } + + protected Instant resolveTimestampStringAsInstant(String dateText) { + LocalDateTime dateTime; + if (dateText.indexOf(" AM") > 0 || dateText.indexOf(" PM") > 0) { + dateTime = LocalDateTime.from(TIMESTAMP_AM_PM_SHORT_FORMATTER.parse(dateText.trim())); + } else { + dateTime = LocalDateTime.from(TIMESTAMP_FORMATTER.parse(dateText.trim())); + } + return dateTime.atZone(ZoneId.of(serverTimezone)).toInstant(); + } + + @Override + protected Object convertString(Column column, Field fieldDefn, Object data) { + if (data instanceof Clob) { + try { + Clob clob = (Clob) data; + return clob.getSubString(1, (int) clob.length()); + } catch (SQLException e) { + throw new RuntimeException("Couldn't convert value for column " + column.name(), e); + } + } + if (data instanceof String) { + String s = (String) data; + if (EMPTY_CLOB_FUNCTION.equals(s)) { + return column.isOptional() ? null : ""; + } + } + return super.convertString(column, fieldDefn, data); + } + + @Override + protected Object convertBinary( + Column column, + Field fieldDefn, + Object data, + CommonConnectorConfig.BinaryHandlingMode mode) { + try { + if (data instanceof Blob) { + Blob blob = (Blob) data; + data = blob.getBytes(1, Long.valueOf(blob.length()).intValue()); + } + if (data instanceof String) { + String str = (String) data; + if (EMPTY_BLOB_FUNCTION.equals(str)) { + data = column.isOptional() ? null : ""; + } + } + return super.convertBinary(column, fieldDefn, data, mode); + } catch (SQLException e) { + throw new RuntimeException("Couldn't convert value for column " + column.name(), e); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java deleted file mode 100644 index 9b36fce8c76..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/RowDataOceanBaseDeserializationSchema.java +++ /dev/null @@ -1,632 +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.source; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseAppendMetadataCollector; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseDeserializationSchema; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseMetadataConverter; -import org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseRecord; -import org.apache.flink.cdc.debezium.utils.TemporalConversions; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; - -import com.oceanbase.oms.logmessage.ByteString; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.Arrays; -import java.util.Map; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Deserialization schema from OceanBase object to Flink Table/SQL internal data structure {@link - * RowData}. - */ -public class RowDataOceanBaseDeserializationSchema - implements OceanBaseDeserializationSchema { - - private static final long serialVersionUID = 1L; - - /** TypeInformation of the produced {@link RowData}. * */ - private final TypeInformation resultTypeInfo; - - /** - * Runtime converter that OceanBase record data into {@link RowData} consisted of physical - * column values. - */ - private final OceanBaseDeserializationRuntimeConverter physicalConverter; - - /** Whether the deserializer needs to handle metadata columns. */ - private final boolean hasMetadata; - - /** - * A wrapped output collector which is used to append metadata columns after physical columns. - */ - private final OceanBaseAppendMetadataCollector appendMetadataCollector; - - /** Returns a builder to build {@link RowDataOceanBaseDeserializationSchema}. */ - public static RowDataOceanBaseDeserializationSchema.Builder newBuilder() { - return new RowDataOceanBaseDeserializationSchema.Builder(); - } - - RowDataOceanBaseDeserializationSchema( - RowType physicalDataType, - OceanBaseMetadataConverter[] metadataConverters, - TypeInformation resultTypeInfo, - ZoneId serverTimeZone) { - this.hasMetadata = checkNotNull(metadataConverters).length > 0; - this.appendMetadataCollector = new OceanBaseAppendMetadataCollector(metadataConverters); - this.physicalConverter = createConverter(checkNotNull(physicalDataType), serverTimeZone); - this.resultTypeInfo = checkNotNull(resultTypeInfo); - } - - @Override - public void deserialize(OceanBaseRecord record, Collector out) throws Exception { - RowData physicalRow; - if (record.isSnapshotRecord()) { - physicalRow = (GenericRowData) physicalConverter.convert(record.getJdbcFields()); - physicalRow.setRowKind(RowKind.INSERT); - emit(record, physicalRow, out); - } else { - switch (record.getOpt()) { - case INSERT: - physicalRow = - (GenericRowData) - physicalConverter.convert(record.getLogMessageFieldsAfter()); - physicalRow.setRowKind(RowKind.INSERT); - emit(record, physicalRow, out); - break; - case DELETE: - physicalRow = - (GenericRowData) - physicalConverter.convert(record.getLogMessageFieldsBefore()); - physicalRow.setRowKind(RowKind.DELETE); - emit(record, physicalRow, out); - break; - case UPDATE: - physicalRow = - (GenericRowData) - physicalConverter.convert(record.getLogMessageFieldsBefore()); - physicalRow.setRowKind(RowKind.UPDATE_BEFORE); - emit(record, physicalRow, out); - physicalRow = - (GenericRowData) - physicalConverter.convert(record.getLogMessageFieldsAfter()); - physicalRow.setRowKind(RowKind.UPDATE_AFTER); - emit(record, physicalRow, out); - break; - default: - throw new IllegalArgumentException( - "Unsupported log message record type: " + record.getOpt()); - } - } - } - - private void emit(OceanBaseRecord row, RowData physicalRow, Collector collector) { - if (!hasMetadata) { - collector.collect(physicalRow); - return; - } - - appendMetadataCollector.inputRecord = row; - appendMetadataCollector.outputCollector = collector; - appendMetadataCollector.collect(physicalRow); - } - - @Override - public TypeInformation getProducedType() { - return resultTypeInfo; - } - - /** Builder class of {@link RowDataOceanBaseDeserializationSchema}. */ - public static class Builder { - private RowType physicalRowType; - private TypeInformation resultTypeInfo; - private OceanBaseMetadataConverter[] metadataConverters = new OceanBaseMetadataConverter[0]; - private ZoneId serverTimeZone = ZoneId.of("UTC"); - - public RowDataOceanBaseDeserializationSchema.Builder setPhysicalRowType( - RowType physicalRowType) { - this.physicalRowType = physicalRowType; - return this; - } - - public RowDataOceanBaseDeserializationSchema.Builder setMetadataConverters( - OceanBaseMetadataConverter[] metadataConverters) { - this.metadataConverters = metadataConverters; - return this; - } - - public RowDataOceanBaseDeserializationSchema.Builder setResultTypeInfo( - TypeInformation resultTypeInfo) { - this.resultTypeInfo = resultTypeInfo; - return this; - } - - public RowDataOceanBaseDeserializationSchema.Builder setServerTimeZone( - ZoneId serverTimeZone) { - this.serverTimeZone = serverTimeZone; - return this; - } - - public RowDataOceanBaseDeserializationSchema build() { - return new RowDataOceanBaseDeserializationSchema( - physicalRowType, metadataConverters, resultTypeInfo, serverTimeZone); - } - } - - private static OceanBaseDeserializationRuntimeConverter createConverter( - LogicalType type, ZoneId serverTimeZone) { - return wrapIntoNullableConverter(createNotNullConverter(type, serverTimeZone)); - } - - private static OceanBaseDeserializationRuntimeConverter wrapIntoNullableConverter( - OceanBaseDeserializationRuntimeConverter converter) { - return new OceanBaseDeserializationRuntimeConverter() { - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) throws Exception { - if (object == null) { - return null; - } - return converter.convert(object); - } - }; - } - - public static OceanBaseDeserializationRuntimeConverter createNotNullConverter( - LogicalType type, ZoneId serverTimeZone) { - switch (type.getTypeRoot()) { - case ROW: - return createRowConverter((RowType) type, serverTimeZone); - case NULL: - return convertToNull(); - case BOOLEAN: - return convertToBoolean(); - case TINYINT: - return convertToTinyInt(); - case SMALLINT: - return convertToSmallInt(); - case INTEGER: - case INTERVAL_YEAR_MONTH: - return convertToInt(); - case BIGINT: - case INTERVAL_DAY_TIME: - return convertToLong(); - case DATE: - return convertToDate(); - case TIME_WITHOUT_TIME_ZONE: - return convertToTime(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return convertToTimestamp(); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return convertToLocalTimeZoneTimestamp(serverTimeZone); - case FLOAT: - return convertToFloat(); - case DOUBLE: - return convertToDouble(); - case CHAR: - case VARCHAR: - return convertToString(); - case BINARY: - return convertToBinary(); - case VARBINARY: - return convertToBytes(); - case DECIMAL: - return createDecimalConverter((DecimalType) type); - case ARRAY: - return createArrayConverter(); - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private static OceanBaseDeserializationRuntimeConverter createRowConverter( - RowType rowType, ZoneId serverTimeZone) { - final OceanBaseDeserializationRuntimeConverter[] fieldConverters = - rowType.getFields().stream() - .map(RowType.RowField::getType) - .map(logicType -> createConverter(logicType, serverTimeZone)) - .toArray(OceanBaseDeserializationRuntimeConverter[]::new); - final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - int arity = fieldNames.length; - GenericRowData row = new GenericRowData(arity); - Map fieldMap = (Map) object; - for (int i = 0; i < arity; i++) { - String fieldName = fieldNames[i]; - Object value = fieldMap.get(fieldName); - try { - row.setField(i, fieldConverters[i].convert(value)); - } catch (Exception e) { - throw new RuntimeException( - "Failed to convert field '" + fieldName + "' with value: " + value, - e); - } - } - return row; - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToNull() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - return null; - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToBoolean() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof byte[]) { - return "1".equals(new String((byte[]) object, StandardCharsets.UTF_8)); - } - return Boolean.parseBoolean(object.toString()) || "1".equals(object.toString()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToTinyInt() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - return Byte.parseByte(object.toString()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToSmallInt() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - return Short.parseShort(object.toString()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToInt() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof Integer) { - return object; - } else if (object instanceof Long) { - return ((Long) object).intValue(); - } else if (object instanceof Date) { - return ((Date) object).toLocalDate().getYear(); - } else { - return Integer.parseInt(object.toString()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToLong() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof Integer) { - return ((Integer) object).longValue(); - } else if (object instanceof Long) { - return object; - } else { - return Long.parseLong(object.toString()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToDouble() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof Float) { - return ((Float) object).doubleValue(); - } else if (object instanceof Double) { - return object; - } else { - return Double.parseDouble(object.toString()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToFloat() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof Float) { - return object; - } else if (object instanceof Double) { - return ((Double) object).floatValue(); - } else { - return Float.parseFloat(object.toString()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToDate() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof String) { - object = Date.valueOf((String) object); - } - return (int) TemporalConversions.toLocalDate(object).toEpochDay(); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToTime() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof Long) { - return (int) ((Long) object / 1000_000); - } - if (object instanceof String) { - object = Time.valueOf((String) object); - } - return TemporalConversions.toLocalTime(object).toSecondOfDay() * 1000; - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToTimestamp() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof String) { - object = Timestamp.valueOf((String) object); - } - if (object instanceof Timestamp) { - return TimestampData.fromTimestamp((Timestamp) object); - } - if (object instanceof LocalDateTime) { - return TimestampData.fromLocalDateTime((LocalDateTime) object); - } - throw new IllegalArgumentException( - "Unable to convert to TimestampData from unexpected value '" - + object - + "' of type " - + object.getClass().getName()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToLocalTimeZoneTimestamp( - ZoneId serverTimeZone) { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof String) { - object = Timestamp.valueOf((String) object); - } - if (object instanceof Timestamp) { - return TimestampData.fromInstant( - ((Timestamp) object) - .toLocalDateTime() - .atZone(serverTimeZone) - .toInstant()); - } - if (object instanceof LocalDateTime) { - return TimestampData.fromInstant( - ((LocalDateTime) object).atZone(serverTimeZone).toInstant()); - } - throw new IllegalArgumentException( - "Unable to convert to TimestampData from unexpected value '" - + object - + "' of type " - + object.getClass().getName()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToString() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - return StringData.fromString(object.toString()); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToBinary() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof String) { - try { - long v = Long.parseLong((String) object); - byte[] bytes = ByteBuffer.allocate(8).putLong(v).array(); - int i = 0; - while (i < Long.BYTES - 1 && bytes[i] == 0) { - i++; - } - return Arrays.copyOfRange(bytes, i, Long.BYTES); - } catch (NumberFormatException e) { - return ((String) object).getBytes(StandardCharsets.UTF_8); - } - } else if (object instanceof byte[]) { - String str = new String((byte[]) object, StandardCharsets.US_ASCII); - return str.getBytes(StandardCharsets.UTF_8); - } else if (object instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) object; - byte[] bytes = new byte[byteBuffer.remaining()]; - byteBuffer.get(bytes); - return bytes; - } else { - throw new UnsupportedOperationException( - "Unsupported BINARY value type: " + object.getClass().getSimpleName()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter convertToBytes() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - if (object instanceof String) { - return ((String) object).getBytes(StandardCharsets.UTF_8); - } else if (object instanceof byte[]) { - return object; - } else if (object instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) object; - byte[] bytes = new byte[byteBuffer.remaining()]; - byteBuffer.get(bytes); - return bytes; - } else { - throw new UnsupportedOperationException( - "Unsupported BYTES value type: " + object.getClass().getSimpleName()); - } - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter createDecimalConverter( - DecimalType decimalType) { - final int precision = decimalType.getPrecision(); - final int scale = decimalType.getScale(); - - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - BigDecimal bigDecimal; - if (object instanceof String) { - bigDecimal = new BigDecimal((String) object); - } else if (object instanceof Long) { - bigDecimal = new BigDecimal((Long) object); - } else if (object instanceof BigInteger) { - bigDecimal = new BigDecimal((BigInteger) object); - } else if (object instanceof Double) { - bigDecimal = BigDecimal.valueOf((Double) object); - } else if (object instanceof BigDecimal) { - bigDecimal = (BigDecimal) object; - } else { - throw new IllegalArgumentException( - "Unable to convert to decimal from unexpected value '" - + object - + "' of type " - + object.getClass()); - } - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - } - }; - } - - private static OceanBaseDeserializationRuntimeConverter createArrayConverter() { - return new OceanBaseDeserializationRuntimeConverter() { - - private static final long serialVersionUID = 1L; - - @Override - public Object convert(Object object) { - String s; - if (object instanceof ByteString) { - s = ((ByteString) object).toString(StandardCharsets.UTF_8.name()); - } else { - s = object.toString(); - } - String[] strArray = s.split(","); - StringData[] stringDataArray = new StringData[strArray.length]; - for (int i = 0; i < strArray.length; i++) { - stringDataArray[i] = StringData.fromString(strArray[i]); - } - return new GenericArrayData(stringDataArray); - } - }; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java new file mode 100644 index 00000000000..81f4399fd62 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java @@ -0,0 +1,98 @@ +/* + * 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.source.config; + +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSourceInfoStructMaker; + +import io.debezium.config.Configuration; +import io.debezium.connector.SourceInfoStructMaker; +import io.debezium.relational.ColumnFilterMode; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +/** Debezium connector config. */ +public class OceanBaseConnectorConfig extends RelationalDatabaseConnectorConfig { + + protected static final String LOGICAL_NAME = "oceanbase_cdc_connector"; + protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = Integer.MIN_VALUE; + protected static final List BUILT_IN_DB_NAMES = + Collections.unmodifiableList( + Arrays.asList( + "information_schema", + "mysql", + "oceanbase", + "LBACSYS", + "ORAAUDITOR", + "SYS")); + + private final String compatibleMode; + private final String serverTimeZone; + private final String tenantName; + + public OceanBaseConnectorConfig( + String compatibleMode, + String serverTimeZone, + String tenantName, + Properties properties) { + super( + Configuration.from(properties), + LOGICAL_NAME, + Tables.TableFilter.fromPredicate( + tableId -> + "mysql".equalsIgnoreCase(compatibleMode) + ? !BUILT_IN_DB_NAMES.contains(tableId.catalog()) + : !BUILT_IN_DB_NAMES.contains(tableId.schema())), + TableId::identifier, + DEFAULT_SNAPSHOT_FETCH_SIZE, + "mysql".equalsIgnoreCase(compatibleMode) + ? ColumnFilterMode.CATALOG + : ColumnFilterMode.SCHEMA); + this.compatibleMode = compatibleMode; + this.serverTimeZone = serverTimeZone; + this.tenantName = tenantName; + } + + public String getCompatibleMode() { + return compatibleMode; + } + + public String getServerTimeZone() { + return serverTimeZone; + } + + @Override + public String getConnectorName() { + return "oceanbase"; + } + + @Override + public String getContextName() { + return "OceanBase"; + } + + @Override + protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { + return new OceanBaseSourceInfoStructMaker(tenantName); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java new file mode 100644 index 00000000000..f7a4eb6cd29 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.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.source.schema; + +import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseValueConverters; +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.relational.RelationalDatabaseSchema; +import io.debezium.relational.TableSchemaBuilder; +import io.debezium.schema.TopicSelector; + +/** OceanBase database schema. */ +public class OceanBaseDatabaseSchema extends RelationalDatabaseSchema { + + public OceanBaseDatabaseSchema( + OceanBaseConnectorConfig connectorConfig, boolean tableIdCaseInsensitive) { + super( + connectorConfig, + TopicSelector.defaultSelector( + connectorConfig, + (tableId, prefix, delimiter) -> + String.join(delimiter, prefix, tableId.identifier())), + connectorConfig.getTableFilters().dataCollectionFilter(), + connectorConfig.getColumnFilter(), + new TableSchemaBuilder( + new OceanBaseValueConverters(connectorConfig), + connectorConfig.schemaNameAdjustmentMode().createAdjuster(), + connectorConfig.customConverterRegistry(), + connectorConfig.getSourceInfoStructMaker().schema(), + connectorConfig.getSanitizeFieldNames(), + false), + tableIdCaseInsensitive, + connectorConfig.getKeyMapper()); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java new file mode 100644 index 00000000000..70b525ec003 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfo.java @@ -0,0 +1,94 @@ +/* + * 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.source.schema; + +import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig; + +import io.debezium.connector.common.BaseSourceInfo; +import io.debezium.relational.TableId; + +import java.time.Instant; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** OceanBase source info. */ +public class OceanBaseSourceInfo extends BaseSourceInfo { + + public static final String TENANT_KEY = "tenant"; + public static final String TRANSACTION_ID_KEY = "transaction_id"; + + private Instant sourceTime; + private Set tableIds; + private String transactionId; + + public OceanBaseSourceInfo(OceanBaseConnectorConfig config) { + super(config); + } + + @Override + protected Instant timestamp() { + return sourceTime; + } + + public void setSourceTime(Instant sourceTime) { + this.sourceTime = sourceTime; + } + + public void beginTransaction(String transactionId) { + this.transactionId = transactionId; + } + + public void commitTransaction() { + this.transactionId = null; + } + + public String transactionId() { + return transactionId; + } + + public void tableEvent(TableId tableId) { + this.tableIds = Collections.singleton(tableId); + } + + @Override + protected String database() { + return (tableIds != null) ? tableIds.iterator().next().catalog() : null; + } + + public String tableSchema() { + return (tableIds == null || tableIds.isEmpty()) + ? null + : tableIds.stream() + .filter(Objects::nonNull) + .map(TableId::schema) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.joining(",")); + } + + public String table() { + return (tableIds == null || tableIds.isEmpty()) + ? null + : tableIds.stream() + .filter(Objects::nonNull) + .map(TableId::table) + .collect(Collectors.joining(",")); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java new file mode 100644 index 00000000000..901e136686b --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSourceInfoStructMaker.java @@ -0,0 +1,72 @@ +/* + * 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.source.schema; + +import io.debezium.connector.SourceInfoStructMaker; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; + +/** The {@link SourceInfoStructMaker} implementation for OceanBase. */ +public class OceanBaseSourceInfoStructMaker implements SourceInfoStructMaker { + + private final String tenant; + private final Schema schema; + + public OceanBaseSourceInfoStructMaker(String tenant) { + this.tenant = tenant; + this.schema = + SchemaBuilder.struct() + .field(OceanBaseSourceInfo.TABLE_NAME_KEY, Schema.STRING_SCHEMA) + .field(OceanBaseSourceInfo.TIMESTAMP_KEY, Schema.INT64_SCHEMA) + .field(OceanBaseSourceInfo.TENANT_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field(OceanBaseSourceInfo.DATABASE_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field(OceanBaseSourceInfo.SCHEMA_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field( + OceanBaseSourceInfo.TRANSACTION_ID_KEY, + Schema.OPTIONAL_STRING_SCHEMA) + .build(); + } + + @Override + public Schema schema() { + return schema; + } + + @Override + public Struct struct(OceanBaseSourceInfo sourceInfo) { + Struct source = new Struct(schema); + if (tenant != null) { + source.put(OceanBaseSourceInfo.TENANT_KEY, tenant); + } + if (sourceInfo.database() != null) { + source.put(OceanBaseSourceInfo.DATABASE_NAME_KEY, sourceInfo.database()); + } + if (sourceInfo.tableSchema() != null) { + source.put(OceanBaseSourceInfo.SCHEMA_NAME_KEY, sourceInfo.tableSchema()); + } + source.put(OceanBaseSourceInfo.TABLE_NAME_KEY, sourceInfo.table()); + if (sourceInfo.timestamp() != null) { + source.put(OceanBaseSourceInfo.TIMESTAMP_KEY, sourceInfo.timestamp().toEpochMilli()); + } + if (sourceInfo.transactionId() != null) { + source.put(OceanBaseSourceInfo.TRANSACTION_ID_KEY, sourceInfo.transactionId()); + } + return source; + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java deleted file mode 100644 index 72cc5794359..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseAppendMetadataCollector.java +++ /dev/null @@ -1,57 +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.table; - -import org.apache.flink.cdc.common.annotation.Internal; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.utils.JoinedRowData; -import org.apache.flink.util.Collector; - -import java.io.Serializable; - -/** Emits a row with physical fields and metadata fields. */ -@Internal -public class OceanBaseAppendMetadataCollector implements Collector, Serializable { - private static final long serialVersionUID = 1L; - - private final OceanBaseMetadataConverter[] metadataConverters; - - public transient OceanBaseRecord inputRecord; - public transient Collector outputCollector; - - public OceanBaseAppendMetadataCollector(OceanBaseMetadataConverter[] metadataConverters) { - this.metadataConverters = metadataConverters; - } - - @Override - public void collect(RowData physicalRow) { - GenericRowData metaRow = new GenericRowData(metadataConverters.length); - for (int i = 0; i < metadataConverters.length; i++) { - Object meta = metadataConverters[i].read(inputRecord); - metaRow.setField(i, meta); - } - RowData outRow = new JoinedRowData(physicalRow.getRowKind(), physicalRow, metaRow); - outputCollector.collect(outRow); - } - - @Override - public void close() { - // nothing to do - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java deleted file mode 100644 index f87208c267b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseDeserializationSchema.java +++ /dev/null @@ -1,37 +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.table; - -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.cdc.common.annotation.PublicEvolving; -import org.apache.flink.util.Collector; - -import java.io.Serializable; - -/** - * The deserialization schema describes how to turn the OceanBase record into data types (Java/Scala - * objects) that are processed by Flink. - * - * @param The type created by the deserialization schema. - */ -@PublicEvolving -public interface OceanBaseDeserializationSchema extends Serializable, ResultTypeQueryable { - - /** Deserialize the OceanBase record, it is represented in {@link OceanBaseRecord}. */ - void deserialize(OceanBaseRecord record, Collector out) throws Exception; -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java deleted file mode 100644 index 953f860c851..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMetadataConverter.java +++ /dev/null @@ -1,29 +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.table; - -import org.apache.flink.cdc.common.annotation.Internal; - -import java.io.Serializable; - -/** A converter converts OceanBase record metadata into Flink internal data structures. */ -@FunctionalInterface -@Internal -public interface OceanBaseMetadataConverter extends Serializable { - Object read(OceanBaseRecord record); -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java index 021465a717d..e1b0015306c 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java @@ -17,37 +17,64 @@ package org.apache.flink.cdc.connectors.oceanbase.table; +import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSourceInfo; +import org.apache.flink.cdc.debezium.table.MetadataConverter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; +import io.debezium.data.Envelope; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + /** Defines the supported metadata columns for {@link OceanBaseTableSource}. */ public enum OceanBaseReadableMetadata { /** Name of the tenant that contains the row. */ TENANT( "tenant_name", - DataTypes.STRING().notNull(), - new OceanBaseMetadataConverter() { + DataTypes.STRING().nullable(), + new MetadataConverter() { private static final long serialVersionUID = 1L; @Override - public Object read(OceanBaseRecord record) { - return StringData.fromString(record.getSourceInfo().getTenant()); + public Object read(SourceRecord record) { + Struct messageStruct = (Struct) record.value(); + Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); + return StringData.fromString( + sourceStruct.getString(OceanBaseSourceInfo.TENANT_KEY)); } }), /** Name of the database that contains the row. */ DATABASE( "database_name", - DataTypes.STRING().notNull(), - new OceanBaseMetadataConverter() { + DataTypes.STRING().nullable(), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object read(SourceRecord record) { + Struct messageStruct = (Struct) record.value(); + Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); + return StringData.fromString( + sourceStruct.getString(OceanBaseSourceInfo.DATABASE_NAME_KEY)); + } + }), + + SCHEMA( + "schema_name", + DataTypes.STRING().nullable(), + new MetadataConverter() { private static final long serialVersionUID = 1L; @Override - public Object read(OceanBaseRecord record) { - return StringData.fromString(record.getSourceInfo().getDatabase()); + public Object read(SourceRecord record) { + Struct messageStruct = (Struct) record.value(); + Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); + return StringData.fromString( + sourceStruct.getString(OceanBaseSourceInfo.SCHEMA_NAME_KEY)); } }), @@ -55,12 +82,15 @@ public Object read(OceanBaseRecord record) { TABLE( "table_name", DataTypes.STRING().notNull(), - new OceanBaseMetadataConverter() { + new MetadataConverter() { private static final long serialVersionUID = 1L; @Override - public Object read(OceanBaseRecord record) { - return StringData.fromString(record.getSourceInfo().getTable()); + public Object read(SourceRecord record) { + Struct messageStruct = (Struct) record.value(); + Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); + return StringData.fromString( + sourceStruct.getString(OceanBaseSourceInfo.TENANT_KEY)); } }), @@ -71,13 +101,15 @@ public Object read(OceanBaseRecord record) { OP_TS( "op_ts", DataTypes.TIMESTAMP_LTZ(3).notNull(), - new OceanBaseMetadataConverter() { + new MetadataConverter() { private static final long serialVersionUID = 1L; @Override - public Object read(OceanBaseRecord record) { + public Object read(SourceRecord record) { + Struct messageStruct = (Struct) record.value(); + Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE); return TimestampData.fromEpochMillis( - record.getSourceInfo().getTimestampS() * 1000); + sourceStruct.getInt64(OceanBaseSourceInfo.TIMESTAMP_KEY)); } }); @@ -85,9 +117,9 @@ public Object read(OceanBaseRecord record) { private final DataType dataType; - private final OceanBaseMetadataConverter converter; + private final MetadataConverter converter; - OceanBaseReadableMetadata(String key, DataType dataType, OceanBaseMetadataConverter converter) { + OceanBaseReadableMetadata(String key, DataType dataType, MetadataConverter converter) { this.key = key; this.dataType = dataType; this.converter = converter; @@ -101,7 +133,7 @@ public DataType getDataType() { return dataType; } - public OceanBaseMetadataConverter getConverter() { + public MetadataConverter getConverter() { return converter; } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java deleted file mode 100644 index 716372c29ed..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseRecord.java +++ /dev/null @@ -1,133 +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.table; - -import com.oceanbase.oms.logmessage.DataMessage; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** An internal data structure representing record of OceanBase. */ -public class OceanBaseRecord implements Serializable { - private static final long serialVersionUID = 1L; - - private final SourceInfo sourceInfo; - private final boolean isSnapshotRecord; - private final Map jdbcFields; - private final DataMessage.Record.Type opt; - private final Map logMessageFieldsBefore; - private final Map logMessageFieldsAfter; - - public OceanBaseRecord(SourceInfo sourceInfo, Map jdbcFields) { - this.sourceInfo = sourceInfo; - this.isSnapshotRecord = true; - this.jdbcFields = jdbcFields; - this.opt = null; - this.logMessageFieldsBefore = null; - this.logMessageFieldsAfter = null; - } - - public OceanBaseRecord( - SourceInfo sourceInfo, - DataMessage.Record.Type opt, - List logMessageFieldList) { - this.sourceInfo = sourceInfo; - this.isSnapshotRecord = false; - this.jdbcFields = null; - this.opt = opt; - this.logMessageFieldsBefore = new HashMap<>(); - this.logMessageFieldsAfter = new HashMap<>(); - for (DataMessage.Record.Field field : logMessageFieldList) { - if (field.isPrev()) { - logMessageFieldsBefore.put(field.getFieldname(), getFieldStringValue(field)); - } else { - logMessageFieldsAfter.put(field.getFieldname(), getFieldStringValue(field)); - } - } - } - - private String getFieldStringValue(DataMessage.Record.Field field) { - if (field.getValue() == null) { - return null; - } - String encoding = field.getEncoding(); - if ("binary".equalsIgnoreCase(encoding)) { - return field.getValue().toString("utf8"); - } - return field.getValue().toString(encoding); - } - - public SourceInfo getSourceInfo() { - return sourceInfo; - } - - public boolean isSnapshotRecord() { - return isSnapshotRecord; - } - - public Map getJdbcFields() { - return jdbcFields; - } - - public DataMessage.Record.Type getOpt() { - return opt; - } - - public Map getLogMessageFieldsBefore() { - return logMessageFieldsBefore; - } - - public Map getLogMessageFieldsAfter() { - return logMessageFieldsAfter; - } - - /** Information about the source of record. */ - public static class SourceInfo implements Serializable { - private static final long serialVersionUID = 1L; - - private final String tenant; - private final String database; - private final String table; - private final long timestampS; - - public SourceInfo(String tenant, String database, String table, long timestampS) { - this.tenant = tenant; - this.database = database; - this.table = table; - this.timestampS = timestampS; - } - - public String getTenant() { - return tenant; - } - - public String getDatabase() { - return database; - } - - public String getTable() { - return table; - } - - public long getTimestampS() { - return timestampS; - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java index 0172e9e6b60..788932917f1 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java @@ -18,8 +18,12 @@ package org.apache.flink.cdc.connectors.oceanbase.table; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource; -import org.apache.flink.cdc.connectors.oceanbase.source.RowDataOceanBaseDeserializationSchema; +import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseDeserializationConverterFactory; +import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.flink.cdc.debezium.table.MetadataConverter; +import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -47,7 +51,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet private final ResolvedSchema physicalSchema; - private final StartupMode startupMode; + private final StartupOptions startupOptions; private final String username; private final String password; private final String tenantName; @@ -71,6 +75,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet private final String configUrl; private final String workingMode; private final Properties obcdcProperties; + private final Properties debeziumProperties; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -84,7 +89,7 @@ public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMet public OceanBaseTableSource( ResolvedSchema physicalSchema, - StartupMode startupMode, + StartupOptions startupOptions, String username, String password, String tenantName, @@ -105,9 +110,10 @@ public OceanBaseTableSource( String rsList, String configUrl, String workingMode, - Properties obcdcProperties) { + Properties obcdcProperties, + Properties debeziumProperties) { this.physicalSchema = physicalSchema; - this.startupMode = checkNotNull(startupMode); + this.startupOptions = checkNotNull(startupOptions); this.username = checkNotNull(username); this.password = checkNotNull(password); this.tenantName = checkNotNull(tenantName); @@ -129,6 +135,7 @@ public OceanBaseTableSource( this.configUrl = configUrl; this.workingMode = workingMode; this.obcdcProperties = obcdcProperties; + this.debeziumProperties = debeziumProperties; this.producedDataType = physicalSchema.toPhysicalRowDataType(); this.metadataKeys = Collections.emptyList(); @@ -143,20 +150,25 @@ public ChangelogMode getChangelogMode() { public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { RowType physicalDataType = (RowType) physicalSchema.toPhysicalRowDataType().getLogicalType(); - OceanBaseMetadataConverter[] metadataConverters = getMetadataConverters(); + MetadataConverter[] metadataConverters = getMetadataConverters(); TypeInformation resultTypeInfo = context.createTypeInformation(producedDataType); - RowDataOceanBaseDeserializationSchema deserializer = - RowDataOceanBaseDeserializationSchema.newBuilder() + DebeziumDeserializationSchema deserializer = + RowDataDebeziumDeserializeSchema.newBuilder() .setPhysicalRowType(physicalDataType) .setMetadataConverters(metadataConverters) .setResultTypeInfo(resultTypeInfo) - .setServerTimeZone(ZoneId.of(serverTimeZone)) + .setServerTimeZone( + serverTimeZone == null + ? ZoneId.systemDefault() + : ZoneId.of(serverTimeZone)) + .setUserDefinedConverterFactory( + OceanBaseDeserializationConverterFactory.instance()) .build(); OceanBaseSource.Builder builder = OceanBaseSource.builder() - .startupMode(startupMode) + .startupOptions(startupOptions) .username(username) .password(password) .tenantName(tenantName) @@ -178,13 +190,14 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { .configUrl(configUrl) .workingMode(workingMode) .obcdcProperties(obcdcProperties) + .debeziumProperties(debeziumProperties) .deserializer(deserializer); return SourceFunctionProvider.of(builder.build(), false); } - protected OceanBaseMetadataConverter[] getMetadataConverters() { + protected MetadataConverter[] getMetadataConverters() { if (metadataKeys.isEmpty()) { - return new OceanBaseMetadataConverter[0]; + return new MetadataConverter[0]; } return metadataKeys.stream() .map( @@ -194,7 +207,7 @@ protected OceanBaseMetadataConverter[] getMetadataConverters() { .findFirst() .orElseThrow(IllegalStateException::new)) .map(OceanBaseReadableMetadata::getConverter) - .toArray(OceanBaseMetadataConverter[]::new); + .toArray(MetadataConverter[]::new); } @Override @@ -217,7 +230,7 @@ public DynamicTableSource copy() { OceanBaseTableSource source = new OceanBaseTableSource( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -238,7 +251,8 @@ public DynamicTableSource copy() { rsList, configUrl, workingMode, - obcdcProperties); + obcdcProperties, + debeziumProperties); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -254,7 +268,7 @@ public boolean equals(Object o) { } OceanBaseTableSource that = (OceanBaseTableSource) o; return Objects.equals(this.physicalSchema, that.physicalSchema) - && Objects.equals(this.startupMode, that.startupMode) + && Objects.equals(this.startupOptions, that.startupOptions) && Objects.equals(this.username, that.username) && Objects.equals(this.password, that.password) && Objects.equals(this.tenantName, that.tenantName) @@ -276,6 +290,7 @@ public boolean equals(Object o) { && Objects.equals(this.configUrl, that.configUrl) && Objects.equals(this.workingMode, that.workingMode) && Objects.equals(this.obcdcProperties, that.obcdcProperties) + && Objects.equals(this.debeziumProperties, that.debeziumProperties) && Objects.equals(this.producedDataType, that.producedDataType) && Objects.equals(this.metadataKeys, that.metadataKeys); } @@ -284,7 +299,7 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -306,6 +321,7 @@ public int hashCode() { configUrl, workingMode, obcdcProperties, + debeziumProperties, producedDataType, metadataKeys); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java index 2de2a4b2c84..14743f05113 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java @@ -17,12 +17,16 @@ package org.apache.flink.cdc.connectors.oceanbase.table; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; +import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils; import org.apache.flink.cdc.connectors.oceanbase.utils.OptionUtils; +import org.apache.flink.cdc.debezium.table.DebeziumOptions; import org.apache.flink.cdc.debezium.utils.JdbcUrlUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSourceFactory; @@ -35,19 +39,13 @@ import java.util.Properties; import java.util.Set; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE; + /** Factory for creating configured instance of {@link OceanBaseTableSource}. */ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory { private static final String IDENTIFIER = "oceanbase-cdc"; - public static final ConfigOption SCAN_STARTUP_MODE = - ConfigOptions.key("scan.startup.mode") - .stringType() - .noDefaultValue() - .withDescription( - "Optional startup mode for OceanBase CDC consumer, valid enumerations are " - + "\"initial\", \"latest-offset\" or \"timestamp\""); - public static final ConfigOption USERNAME = ConfigOptions.key("username") .stringType() @@ -181,14 +179,16 @@ public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory { public DynamicTableSource createDynamicTableSource(Context context) { final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); - helper.validateExcept(JdbcUrlUtils.PROPERTIES_PREFIX, OBCDC_PROPERTIES_PREFIX); + helper.validateExcept( + JdbcUrlUtils.PROPERTIES_PREFIX, + OBCDC_PROPERTIES_PREFIX, + DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX); ResolvedSchema physicalSchema = context.getCatalogTable().getResolvedSchema(); ReadableConfig config = helper.getOptions(); - validate(config); - StartupMode startupMode = StartupMode.getStartupMode(config.get(SCAN_STARTUP_MODE)); + StartupOptions startupOptions = getStartupOptions(config); String username = config.get(USERNAME); String password = config.get(PASSWORD); @@ -205,6 +205,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { String compatibleMode = config.get(COMPATIBLE_MODE); String jdbcDriver = config.get(JDBC_DRIVER); + validateJdbcDriver(compatibleMode, jdbcDriver); + String logProxyHost = config.get(LOG_PROXY_HOST); Integer logProxyPort = config.get(LOG_PROXY_PORT); String logProxyClientId = config.get(LOG_PROXY_CLIENT_ID); @@ -217,7 +219,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new OceanBaseTableSource( physicalSchema, - startupMode, + startupOptions, username, password, tenantName, @@ -238,7 +240,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { rsList, configUrl, workingMode, - getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX)); + getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX), + DebeziumOptions.getDebeziumProperties(context.getCatalogTable().getOptions())); } @Override @@ -249,28 +252,28 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { Set> options = new HashSet<>(); - options.add(SCAN_STARTUP_MODE); options.add(USERNAME); options.add(PASSWORD); - options.add(TENANT_NAME); - options.add(LOG_PROXY_HOST); - options.add(LOG_PROXY_PORT); + options.add(HOSTNAME); + options.add(PORT); return options; } @Override public Set> optionalOptions() { Set> options = new HashSet<>(); + options.add(SCAN_STARTUP_MODE); options.add(SCAN_STARTUP_TIMESTAMP); options.add(DATABASE_NAME); options.add(TABLE_NAME); options.add(TABLE_LIST); - options.add(HOSTNAME); - options.add(PORT); options.add(COMPATIBLE_MODE); options.add(JDBC_DRIVER); options.add(CONNECT_TIMEOUT); options.add(SERVER_TIME_ZONE); + options.add(TENANT_NAME); + options.add(LOG_PROXY_HOST); + options.add(LOG_PROXY_PORT); options.add(LOG_PROXY_CLIENT_ID); options.add(RS_LIST); options.add(CONFIG_URL); @@ -278,26 +281,54 @@ public Set> optionalOptions() { return options; } - private void validate(ReadableConfig config) { - String startupMode = config.get(SCAN_STARTUP_MODE); - if (StartupMode.getStartupMode(startupMode).equals(StartupMode.INITIAL)) { - String compatibleMode = - Objects.requireNonNull( - config.get(COMPATIBLE_MODE), - "'compatible-mode' is required for 'initial' startup mode."); - String jdbcDriver = - Objects.requireNonNull( - config.get(JDBC_DRIVER), - "'jdbc.driver' is required for 'initial' startup mode."); - if (compatibleMode.equalsIgnoreCase("oracle")) { - if (!jdbcDriver.toLowerCase().contains("oceanbase")) { - throw new IllegalArgumentException( - "OceanBase JDBC driver is required for OceanBase Enterprise Edition."); + private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; + private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; + private static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp"; + + private static StartupOptions getStartupOptions(ReadableConfig config) { + String modeString = config.get(SCAN_STARTUP_MODE); + + switch (modeString.toLowerCase()) { + case SCAN_STARTUP_MODE_VALUE_INITIAL: + return StartupOptions.initial(); + case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: + return StartupOptions.snapshot(); + case SCAN_STARTUP_MODE_VALUE_LATEST: + return StartupOptions.latest(); + case SCAN_STARTUP_MODE_VALUE_TIMESTAMP: + if (config.get(SCAN_STARTUP_TIMESTAMP) != null) { + return StartupOptions.timestamp(config.get(SCAN_STARTUP_TIMESTAMP) * 1000); } - Objects.requireNonNull( - config.get(CONFIG_URL), - "'config-url' is required for OceanBase Enterprise Edition."); - } + throw new ValidationException( + String.format( + "Option '%s' should not be empty", SCAN_STARTUP_TIMESTAMP.key())); + + default: + throw new ValidationException( + String.format( + "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s], but was: %s", + SCAN_STARTUP_MODE.key(), + SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_SNAPSHOT, + SCAN_STARTUP_MODE_VALUE_LATEST, + SCAN_STARTUP_MODE_VALUE_TIMESTAMP, + modeString)); + } + } + + private void validateJdbcDriver(String compatibleMode, String jdbcDriver) { + Objects.requireNonNull(compatibleMode, "'compatible-mode' is required."); + Objects.requireNonNull(jdbcDriver, "'jdbc.driver' is required."); + if ("oracle".equalsIgnoreCase(compatibleMode) + && !OceanBaseUtils.isOceanBaseDriver(jdbcDriver)) { + throw new IllegalArgumentException( + "OceanBase JDBC driver is required for OceanBase Oracle mode."); + } + try { + Class.forName(jdbcDriver); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Jdbc driver class not found", e); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java deleted file mode 100644 index a01246b3c8b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/StartupMode.java +++ /dev/null @@ -1,55 +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.table; - -import org.apache.flink.table.api.ValidationException; - -/** Startup modes for the OceanBase CDC Consumer. */ -public enum StartupMode { - /** - * Performs an initial snapshot on the monitored database tables upon first startup, and - * continue to read the commit log. - */ - INITIAL, - - /** - * Never to perform snapshot on the monitored database tables upon first startup, just read from - * the end of the commit log which means only have the changes since the connector was started. - */ - LATEST_OFFSET, - - /** - * Never to perform snapshot on the monitored database tables upon first startup, and directly - * read commit log from the specified timestamp. - */ - TIMESTAMP; - - public static StartupMode getStartupMode(String modeString) { - switch (modeString.toLowerCase()) { - case "initial": - return INITIAL; - case "latest-offset": - return LATEST_OFFSET; - case "timestamp": - return TIMESTAMP; - default: - throw new ValidationException( - String.format("Invalid startup mode '%s'.", modeString)); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java similarity index 69% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java index f114657051b..7cb738b68b7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseDeserializationRuntimeConverter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.flink.cdc.connectors.oceanbase.source; +package org.apache.flink.cdc.connectors.oceanbase.utils; -import java.io.Serializable; +/** Utils for OceanBase. */ +public class OceanBaseUtils { -/** - * Runtime converter that converts objects of OceanBase into objects of Flink Table & SQL internal - * data structures. - */ -public interface OceanBaseDeserializationRuntimeConverter extends Serializable { - - Object convert(Object object) throws Exception; + public static boolean isOceanBaseDriver(String driverClass) { + return "com.oceanbase.jdbc.Driver".equals(driverClass) + || "com.alipay.oceanbase.jdbc.Driver".equals(driverClass); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java index 1aa96221500..ce3b77e66c3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java @@ -17,171 +17,105 @@ package org.apache.flink.cdc.connectors.oceanbase; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.table.utils.LegacyRowResource; +import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.util.TestLogger; -import org.awaitility.Awaitility; -import org.awaitility.core.ConditionTimeoutException; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.ClassRule; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.containers.wait.strategy.Wait; -import org.testcontainers.lifecycle.Startables; +import org.junit.Rule; import java.net.URL; import java.nio.file.Files; import java.nio.file.Paths; import java.sql.Connection; -import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; -import java.time.Duration; import java.util.Arrays; import java.util.List; -import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; /** Basic class for testing OceanBase source. */ -public class OceanBaseTestBase extends TestLogger { - - private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestBase.class); +public abstract class OceanBaseTestBase extends TestLogger { private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - private static final Duration CONTAINER_STARTUP_TIMEOUT = Duration.ofMinutes(4); - - public static final String NETWORK_MODE = "host"; - - // -------------------------------------------------------------------------------------------- - // Attributes about host and port when network is on 'host' mode. - // -------------------------------------------------------------------------------------------- - - protected static int getObServerSqlPort() { - return 2881; - } - - protected static int getLogProxyPort() { - return 2983; - } - - public static String getRsList() { - return "127.0.0.1:2882:2881"; - } - - // -------------------------------------------------------------------------------------------- - // Attributes about user. - // From OceanBase 4.0.0.0 CE, we can only fetch the commit log of non-sys tenant. - // -------------------------------------------------------------------------------------------- - public static final String OB_SYS_PASSWORD = "pswd"; - - protected static String getTenant() { - return "test"; + protected static final int DEFAULT_PARALLELISM = 4; + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + @ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; + + protected String obServerHost; + protected int obServerPort; + protected String logProxyHost; + protected int logProxyPort; + protected String username; + protected String password; + protected String tenant; + + protected String commonOptionString() { + return " 'connector' = 'oceanbase-cdc'," + + " 'scan.startup.mode' = 'initial'," + + " 'username' = '" + + username + + "'," + + " 'password' = '" + + password + + "'," + + " 'tenant-name' = '" + + tenant + + "'," + + " 'hostname' = '" + + obServerHost + + "'," + + " 'port' = '" + + obServerPort + + "'," + + " 'logproxy.host' = '" + + logProxyHost + + "'," + + " 'logproxy.port' = '" + + logProxyPort + + "'," + + " 'compatible-mode' = '" + + compatibleMode() + + "'," + + " 'working-mode' = 'memory',"; } - protected static String getUsername() { - return "root@" + getTenant(); - } + protected abstract String compatibleMode(); - protected static String getPassword() { - return "test"; - } + protected abstract Connection getJdbcConnection() throws SQLException; - @ClassRule - public static final GenericContainer OB_SERVER = - new GenericContainer<>("oceanbase/oceanbase-ce:4.2.0.0") - .withNetworkMode(NETWORK_MODE) - .withEnv("MODE", "slim") - .withEnv("OB_ROOT_PASSWORD", OB_SYS_PASSWORD) - .waitingFor(Wait.forLogMessage(".*boot success!.*", 1)) - .withStartupTimeout(CONTAINER_STARTUP_TIMEOUT) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - @ClassRule - public static final GenericContainer LOG_PROXY = - new GenericContainer<>("whhe/oblogproxy:1.1.3_4x") - .withNetworkMode(NETWORK_MODE) - .withEnv("OB_SYS_PASSWORD", OB_SYS_PASSWORD) - .waitingFor(Wait.forLogMessage(".*boot success!.*", 1)) - .withStartupTimeout(CONTAINER_STARTUP_TIMEOUT) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - @BeforeClass - public static void startContainers() { - LOG.info("Starting containers..."); - Startables.deepStart(Stream.of(OB_SERVER, LOG_PROXY)).join(); - LOG.info("Containers are started."); - - try (Connection connection = - DriverManager.getConnection(getJdbcUrl(""), getUsername(), ""); + protected void setGlobalTimeZone(String serverTimeZone) throws SQLException { + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { - statement.execute(String.format("ALTER USER root IDENTIFIED BY '%s'", getPassword())); - } catch (SQLException e) { - LOG.error("Set test user password failed.", e); - throw new RuntimeException(e); - } - } - - @AfterClass - public static void stopContainers() { - LOG.info("Stopping containers..."); - Stream.of(OB_SERVER, LOG_PROXY).forEach(GenericContainer::stop); - LOG.info("Containers are stopped."); - } - - public static String getJdbcUrl(String databaseName) { - return "jdbc:mysql://" - + OB_SERVER.getHost() - + ":" - + getObServerSqlPort() - + "/" - + databaseName - + "?useSSL=false"; - } - - protected static Connection getJdbcConnection(String databaseName) throws SQLException { - return DriverManager.getConnection(getJdbcUrl(databaseName), getUsername(), getPassword()); - } - - private static void dropTestDatabase(Connection connection, String databaseName) { - try { - Awaitility.await(String.format("Dropping database %s", databaseName)) - .atMost(120, TimeUnit.SECONDS) - .until( - () -> { - try { - String sql = - String.format( - "DROP DATABASE IF EXISTS %s", databaseName); - connection.createStatement().execute(sql); - return true; - } catch (SQLException e) { - LOG.warn( - String.format( - "DROP DATABASE %s failed: {}", databaseName), - e.getMessage()); - return false; - } - }); - } catch (ConditionTimeoutException e) { - throw new IllegalStateException("Failed to drop test database", e); + statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone)); } } protected void initializeTable(String sqlFile) { - final String ddlFile = String.format("ddl/%s.sql", sqlFile); + final String ddlFile = String.format("ddl/%s/%s.sql", compatibleMode(), sqlFile); final URL ddlTestFile = getClass().getClassLoader().getResource(ddlFile); assertNotNull("Cannot locate " + ddlFile, ddlTestFile); - try (Connection connection = getJdbcConnection(""); + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { - dropTestDatabase(connection, sqlFile); final List statements = Arrays.stream( Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() @@ -203,4 +137,29 @@ protected void initializeTable(String sqlFile) { throw new RuntimeException(e); } } + + public static void waitForSinkSize(String sinkName, int expectedSize) + throws InterruptedException { + while (sinkSize(sinkName) < expectedSize) { + Thread.sleep(100); + } + } + + public static int sinkSize(String sinkName) { + synchronized (TestValuesTableFactory.class) { + try { + return TestValuesTableFactory.getRawResults(sinkName).size(); + } catch (IllegalArgumentException e) { + // job is not started yet + return 0; + } + } + } + + public static void assertContainsInAnyOrder(List expected, List actual) { + assertTrue(expected != null && actual != null); + assertTrue( + String.format("expected: %s, actual: %s", expected, actual), + actual.containsAll(expected)); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java similarity index 62% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java index 8f6e0d9a2ac..06844148fe7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java @@ -18,62 +18,129 @@ package org.apache.flink.cdc.connectors.oceanbase.table; import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; -import org.apache.flink.table.utils.LegacyRowResource; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.MountableFile; import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; import java.sql.Statement; +import java.time.Duration; import java.time.ZoneId; import java.util.Arrays; import java.util.List; +import java.util.stream.Stream; -import static org.junit.Assert.assertTrue; +/** Integration tests for OceanBase MySQL mode table source. */ +public class OceanBaseMySQLModeITCase extends OceanBaseTestBase { -/** Integration tests for OceanBase change stream event SQL source. */ -public class OceanBaseConnectorITCase extends OceanBaseTestBase { - - private static final int DEFAULT_PARALLELISM = 2; + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseMySQLModeITCase.class); private final StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment() - .setParallelism(DEFAULT_PARALLELISM); + StreamExecutionEnvironment.getExecutionEnvironment(); private final StreamTableEnvironment tEnv = StreamTableEnvironment.create( env, EnvironmentSettings.newInstance().inStreamingMode().build()); - @ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; + private static final String NETWORK_MODE = "host"; + private static final String OB_SYS_PASSWORD = "123456"; + + @ClassRule + public static final GenericContainer OB_SERVER = + new GenericContainer<>("oceanbase/oceanbase-ce:4.2.0.0") + .withNetworkMode(NETWORK_MODE) + .withEnv("MODE", "slim") + .withEnv("OB_ROOT_PASSWORD", OB_SYS_PASSWORD) + .withEnv("OB_DATAFILE_SIZE", "1G") + .withEnv("OB_LOG_DISK_SIZE", "4G") + .withCopyFileToContainer( + MountableFile.forClasspathResource("ddl/mysql/docker_init.sql"), + "/root/boot/init.d/init.sql") + .waitingFor(Wait.forLogMessage(".*boot success!.*", 1)) + .withStartupTimeout(Duration.ofMinutes(4)) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @ClassRule + public static final GenericContainer LOG_PROXY = + new GenericContainer<>("whhe/oblogproxy:1.1.3_4x") + .withNetworkMode(NETWORK_MODE) + .withEnv("OB_SYS_PASSWORD", OB_SYS_PASSWORD) + .waitingFor(Wait.forLogMessage(".*boot success!.*", 1)) + .withStartupTimeout(Duration.ofMinutes(1)) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @BeforeClass + public static void startContainers() { + LOG.info("Starting containers..."); + Startables.deepStart(Stream.of(OB_SERVER, LOG_PROXY)).join(); + LOG.info("Containers are started."); + } + + @AfterClass + public static void stopContainers() { + LOG.info("Stopping containers..."); + Stream.of(OB_SERVER, LOG_PROXY).forEach(GenericContainer::stop); + LOG.info("Containers are stopped."); + } - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); + protected final String rsList; + + public OceanBaseMySQLModeITCase() { + this.obServerHost = "127.0.0.1"; + this.obServerPort = 2881; + this.logProxyHost = "127.0.0.1"; + this.logProxyPort = 2983; + this.username = "root@test"; + this.password = "123456"; + this.tenant = "test"; + this.rsList = "127.0.0.1:2882:2881"; + } @Before public void before() { TestValuesTableFactory.clearAllData(); env.enableCheckpointing(1000); - env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.AT_LEAST_ONCE); env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500); } + @Override + protected String compatibleMode() { + return "mysql"; + } + + @Override + protected Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + "jdbc:mysql://" + obServerHost + ":" + obServerPort + "/?useSSL=false", + username, + password); + } + + @Override + protected String commonOptionString() { + return super.commonOptionString() + + " 'rootserver-list' = '" + + rsList + + "'," + + " 'jdbc.properties.useSSL' = 'false',"; + } + @Test public void testTableList() throws Exception { initializeTable("inventory"); @@ -87,29 +154,10 @@ public void testTableList() throws Exception { + " weight DECIMAL(20, 10)," + " PRIMARY KEY (`id`) NOT ENFORCED" + ") WITH (" - + " 'connector' = 'oceanbase-cdc'," - + " 'scan.startup.mode' = 'initial'," - + " 'username' = '%s'," - + " 'password' = '%s'," - + " 'tenant-name' = '%s'," - + " 'table-list' = '%s'," - + " 'hostname' = '%s'," - + " 'port' = '%s'," - + " 'logproxy.host' = '%s'," - + " 'logproxy.port' = '%s'," - + " 'rootserver-list' = '%s'," - + " 'working-mode' = 'memory'," - + " 'jdbc.properties.useSSL' = 'false'" + + commonOptionString() + + " 'table-list' = '%s'" + ")", - getUsername(), - getPassword(), - getTenant(), - "inventory.products", - OB_SERVER.getHost(), - getObServerSqlPort(), - LOG_PROXY.getHost(), - getLogProxyPort(), - getRsList()); + "inventory.products"); String sinkDDL = "CREATE TABLE sink (" @@ -132,19 +180,19 @@ public void testTableList() throws Exception { waitForSinkSize("sink", 9); int snapshotSize = sinkSize("sink"); - try (Connection connection = getJdbcConnection("inventory"); + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { statement.execute( - "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); - statement.execute("UPDATE products SET weight='5.1' WHERE id=107;"); + "UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;"); + statement.execute("UPDATE inventory.products SET weight='5.1' WHERE id=107;"); statement.execute( - "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110 + "INSERT INTO inventory.products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110 statement.execute( - "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);"); + "INSERT INTO inventory.products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);"); statement.execute( - "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); - statement.execute("UPDATE products SET weight='5.17' WHERE id=111;"); - statement.execute("DELETE FROM products WHERE id=111;"); + "UPDATE inventory.products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); + statement.execute("UPDATE inventory.products SET weight='5.17' WHERE id=111;"); + statement.execute("DELETE FROM inventory.products WHERE id=111;"); } waitForSinkSize("sink", snapshotSize + 7); @@ -197,7 +245,7 @@ public void testTableList() throws Exception { @Test public void testMetadataColumns() throws Exception { - initializeTable("inventory_meta"); + initializeTable("inventory"); String sourceDDL = String.format( @@ -211,31 +259,12 @@ public void testMetadataColumns() throws Exception { + " weight DECIMAL(20, 10)," + " PRIMARY KEY (`id`) NOT ENFORCED" + ") WITH (" - + " 'connector' = 'oceanbase-cdc'," - + " 'scan.startup.mode' = 'initial'," - + " 'username' = '%s'," - + " 'password' = '%s'," - + " 'tenant-name' = '%s'," + + commonOptionString() + " 'database-name' = '%s'," - + " 'table-name' = '%s'," - + " 'hostname' = '%s'," - + " 'port' = '%s'," - + " 'logproxy.host' = '%s'," - + " 'logproxy.port' = '%s'," - + " 'rootserver-list' = '%s'," - + " 'working-mode' = 'memory'," - + " 'jdbc.properties.useSSL' = 'false'" + + " 'table-name' = '%s'" + ")", - getUsername(), - getPassword(), - getTenant(), - "^inventory_meta$", - "^products$", - OB_SERVER.getHost(), - getObServerSqlPort(), - LOG_PROXY.getHost(), - getLogProxyPort(), - getRsList()); + "^inventory$", + "^products$"); String sinkDDL = "CREATE TABLE sink (" @@ -261,10 +290,10 @@ public void testMetadataColumns() throws Exception { waitForSinkSize("sink", 9); int snapshotSize = sinkSize("sink"); - try (Connection connection = getJdbcConnection("inventory_meta"); + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { statement.execute( - "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); + "UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;"); } waitForSinkSize("sink", snapshotSize + 1); @@ -272,35 +301,35 @@ public void testMetadataColumns() throws Exception { List expected = Arrays.asList( "+I(" - + getTenant() - + ",inventory_meta,products,101,scooter,Small 2-wheel scooter,3.1400000000)", + + tenant + + ",inventory,products,101,scooter,Small 2-wheel scooter,3.1400000000)", "+I(" - + getTenant() - + ",inventory_meta,products,102,car battery,12V car battery,8.1000000000)", + + tenant + + ",inventory,products,102,car battery,12V car battery,8.1000000000)", "+I(" - + getTenant() - + ",inventory_meta,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", + + tenant + + ",inventory,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)", "+I(" - + getTenant() - + ",inventory_meta,products,104,hammer,12oz carpenter's hammer,0.7500000000)", + + tenant + + ",inventory,products,104,hammer,12oz carpenter's hammer,0.7500000000)", "+I(" - + getTenant() - + ",inventory_meta,products,105,hammer,14oz carpenter's hammer,0.8750000000)", + + tenant + + ",inventory,products,105,hammer,14oz carpenter's hammer,0.8750000000)", "+I(" - + getTenant() - + ",inventory_meta,products,106,hammer,16oz carpenter's hammer,1.0000000000)", + + tenant + + ",inventory,products,106,hammer,16oz carpenter's hammer,1.0000000000)", "+I(" - + getTenant() - + ",inventory_meta,products,107,rocks,box of assorted rocks,5.3000000000)", + + tenant + + ",inventory,products,107,rocks,box of assorted rocks,5.3000000000)", "+I(" - + getTenant() - + ",inventory_meta,products,108,jacket,water resistent black wind breaker,0.1000000000)", + + tenant + + ",inventory,products,108,jacket,water resistent black wind breaker,0.1000000000)", "+I(" - + getTenant() - + ",inventory_meta,products,109,spare tire,24 inch spare tire,22.2000000000)", + + tenant + + ",inventory,products,109,spare tire,24 inch spare tire,22.2000000000)", "+U(" - + getTenant() - + ",inventory_meta,products,106,hammer,18oz carpenter hammer,1.0000000000)"); + + tenant + + ",inventory,products,106,hammer,18oz carpenter hammer,1.0000000000)"); List actual = TestValuesTableFactory.getRawResults("sink"); assertContainsInAnyOrder(expected, actual); result.getJobClient().get().cancel().get(); @@ -309,11 +338,9 @@ public void testMetadataColumns() throws Exception { @Test public void testAllDataTypes() throws Exception { String serverTimeZone = "+00:00"; - try (Connection connection = getJdbcConnection(""); - Statement statement = connection.createStatement()) { - statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone)); - } + setGlobalTimeZone(serverTimeZone); tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone)); + initializeTable("column_type_test"); String sourceDDL = String.format( @@ -343,9 +370,9 @@ public void testAllDataTypes() throws Exception { + " time_c TIME(0),\n" + " datetime3_c TIMESTAMP(3),\n" + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP_LTZ,\n" - + " timestamp3_c TIMESTAMP_LTZ(3),\n" - + " timestamp6_c TIMESTAMP_LTZ(6),\n" + + " timestamp_c TIMESTAMP,\n" + + " timestamp3_c TIMESTAMP(3),\n" + + " timestamp6_c TIMESTAMP(6),\n" + " char_c CHAR(3),\n" + " varchar_c VARCHAR(255),\n" + " file_uuid BINARY(16),\n" @@ -361,34 +388,14 @@ public void testAllDataTypes() throws Exception { + " json_c STRING,\n" + " primary key (`id`) not enforced" + ") WITH (" - + " 'connector' = 'oceanbase-cdc'," - + " 'scan.startup.mode' = 'initial'," - + " 'username' = '%s'," - + " 'password' = '%s'," - + " 'tenant-name' = '%s'," + + commonOptionString() + " 'database-name' = '%s'," + " 'table-name' = '%s'," - + " 'server-time-zone' = '%s'," - + " 'hostname' = '%s'," - + " 'port' = '%s'," - + " 'logproxy.host' = '%s'," - + " 'logproxy.port' = '%s'," - + " 'rootserver-list' = '%s'," - + " 'working-mode' = 'memory'," - + " 'jdbc.properties.useSSL' = 'false'," - + " 'obcdc.properties.sort_trans_participants' = '1'" + + " 'server-time-zone' = '%s'" + ")", - getUsername(), - getPassword(), - getTenant(), "^column_type_test$", "^full_types$", - serverTimeZone, - OB_SERVER.getHost(), - getObServerSqlPort(), - LOG_PROXY.getHost(), - getLogProxyPort(), - getRsList()); + serverTimeZone); String sinkDDL = "CREATE TABLE sink (" + " `id` INT NOT NULL,\n" @@ -421,7 +428,7 @@ public void testAllDataTypes() throws Exception { + " timestamp6_c TIMESTAMP(6),\n" + " char_c CHAR(3),\n" + " varchar_c VARCHAR(255),\n" - + " file_uuid BINARY(16),\n" + + " file_uuid STRING,\n" + " bit_c BINARY(8),\n" + " text_c STRING,\n" + " tiny_blob_c BYTES,\n" @@ -441,23 +448,66 @@ public void testAllDataTypes() throws Exception { tEnv.executeSql(sourceDDL); tEnv.executeSql(sinkDDL); - TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source"); + TableResult result = + tEnv.executeSql( + "INSERT INTO sink SELECT id,\n" + + "bit1_c,\n" + + "tiny1_c,\n" + + "boolean_c,\n" + + "tiny_c,\n" + + "tiny_un_c,\n" + + "small_c ,\n" + + "small_un_c,\n" + + "medium_c,\n" + + "medium_un_c,\n" + + "int11_c,\n" + + "int_c,\n" + + "int_un_c,\n" + + "big_c,\n" + + "big_un_c,\n" + + "real_c,\n" + + "float_c,\n" + + "double_c,\n" + + "decimal_c,\n" + + "numeric_c,\n" + + "big_decimal_c,\n" + + "date_c,\n" + + "time_c,\n" + + "datetime3_c,\n" + + "datetime6_c,\n" + + "timestamp_c,\n" + + "timestamp3_c,\n" + + "timestamp6_c,\n" + + "char_c,\n" + + "varchar_c,\n" + + "TO_BASE64(DECODE(file_uuid, 'UTF-8')),\n" + + "bit_c,\n" + + "text_c,\n" + + "tiny_blob_c,\n" + + "medium_blob_c,\n" + + "blob_c,\n" + + "long_blob_c,\n" + + "year_c,\n" + + "set_c,\n" + + "enum_c,\n" + + "json_c\n" + + " FROM ob_source"); waitForSinkSize("sink", 1); int snapshotSize = sinkSize("sink"); - try (Connection connection = getJdbcConnection("column_type_test"); + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { statement.execute( - "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); + "UPDATE column_type_test.full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); } waitForSinkSize("sink", snapshotSize + 1); List expected = Arrays.asList( - "+I(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,[101, 26, -17, -65, -67, 8, 57, 15, 72, -17, -65, -67, -17, -65, -67, -17, -65, -67, 54, -17, -65, -67, 62, 123, 116, 0],[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})", - "+U(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,[101, 26, -17, -65, -67, 8, 57, 15, 72, -17, -65, -67, -17, -65, -67, -17, -65, -67, 54, -17, -65, -67, 62, 123, 116, 0],[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})"); + "+I(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})", + "+U(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})"); List actual = TestValuesTableFactory.getRawResults("sink"); assertContainsInAnyOrder(expected, actual); @@ -475,11 +525,9 @@ public void testTimezoneShanghai() throws Exception { } public void testTimeDataTypes(String serverTimeZone) throws Exception { - try (Connection connection = getJdbcConnection(""); - Statement statement = connection.createStatement()) { - statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone)); - } + setGlobalTimeZone(serverTimeZone); tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone)); + initializeTable("column_type_test"); String sourceDDL = String.format( @@ -489,36 +537,17 @@ public void testTimeDataTypes(String serverTimeZone) throws Exception { + " time_c TIME(0),\n" + " datetime3_c TIMESTAMP(3),\n" + " datetime6_c TIMESTAMP(6),\n" - + " timestamp_c TIMESTAMP_LTZ,\n" + + " timestamp_c TIMESTAMP,\n" + " primary key (`id`) not enforced" + ") WITH (" - + " 'connector' = 'oceanbase-cdc'," - + " 'scan.startup.mode' = 'initial'," - + " 'username' = '%s'," - + " 'password' = '%s'," - + " 'tenant-name' = '%s'," + + commonOptionString() + " 'database-name' = '%s'," + " 'table-name' = '%s'," - + " 'server-time-zone' = '%s'," - + " 'hostname' = '%s'," - + " 'port' = '%s'," - + " 'logproxy.host' = '%s'," - + " 'logproxy.port' = '%s'," - + " 'rootserver-list' = '%s'," - + " 'working-mode' = 'memory'," - + " 'jdbc.properties.useSSL' = 'false'" + + " 'server-time-zone' = '%s'" + ")", - getUsername(), - getPassword(), - getTenant(), "column_type_test", "full_types", - serverTimeZone, - OB_SERVER.getHost(), - getObServerSqlPort(), - LOG_PROXY.getHost(), - getLogProxyPort(), - getRsList()); + serverTimeZone); String sinkDDL = "CREATE TABLE sink (" @@ -546,10 +575,10 @@ public void testTimeDataTypes(String serverTimeZone) throws Exception { waitForSinkSize("sink", 1); int snapshotSize = sinkSize("sink"); - try (Connection connection = getJdbcConnection("column_type_test"); + try (Connection connection = getJdbcConnection(); Statement statement = connection.createStatement()) { statement.execute( - "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); + "UPDATE column_type_test.full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;"); } waitForSinkSize("sink", snapshotSize + 1); @@ -563,29 +592,4 @@ public void testTimeDataTypes(String serverTimeZone) throws Exception { assertContainsInAnyOrder(expected, actual); result.getJobClient().get().cancel().get(); } - - private static void waitForSinkSize(String sinkName, int expectedSize) - throws InterruptedException { - while (sinkSize(sinkName) < expectedSize) { - Thread.sleep(100); - } - } - - private static int sinkSize(String sinkName) { - synchronized (TestValuesTableFactory.class) { - try { - return TestValuesTableFactory.getRawResults(sinkName).size(); - } catch (IllegalArgumentException e) { - // job is not started yet - return 0; - } - } - } - - public static void assertContainsInAnyOrder(List expected, List actual) { - assertTrue(expected != null && actual != null); - assertTrue( - String.format("expected: %s, actual: %s", expected, actual), - actual.containsAll(expected)); - } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java new file mode 100644 index 00000000000..3cf94087693 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java @@ -0,0 +1,249 @@ +/* + * 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.table; + +import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; + +import org.junit.Ignore; +import org.junit.Test; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.List; + +/** Integration tests for OceanBase Oracle mode table source. */ +@Ignore("Test ignored before oceanbase-xe docker image is available") +public class OceanBaseOracleModeITCase extends OceanBaseTestBase { + + private final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + private final StreamTableEnvironment tEnv = + StreamTableEnvironment.create( + env, EnvironmentSettings.newInstance().inStreamingMode().build()); + + private final String schema; + private final String configUrl; + + public OceanBaseOracleModeITCase() { + this.obServerHost = "127.0.0.1"; + this.obServerPort = 2881; + this.logProxyHost = "127.0.0.1"; + this.logProxyPort = 2983; + this.username = "SYS@test"; + this.password = ""; + this.tenant = "test"; + this.schema = "SYS"; + this.configUrl = + "http://127.0.0.1:8080/services?Action=ObRootServiceInfo&User_ID=${User_ID}&UID=${UID}&ObRegion=${ObRegion}"; + } + + @Override + protected String compatibleMode() { + return "oracle"; + } + + @Override + protected Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + "jdbc:oceanbase://" + obServerHost + ":" + obServerPort + "/" + schema, + username, + password); + } + + @Override + protected String commonOptionString() { + return super.commonOptionString() + + " 'config-url' = '" + + configUrl + + "'," + + " 'jdbc.driver' = 'com.oceanbase.jdbc.Driver'," + + " 'connect.timeout' = '2M',"; + } + + @Test + public void testAllDataTypes() throws Exception { + initializeTable("column_type_test"); + String sourceDDL = + String.format( + "CREATE TABLE full_types (" + + " ID INT NOT NULL," + + " VAL_VARCHAR STRING," + + " VAL_VARCHAR2 STRING," + + " VAL_NVARCHAR2 STRING," + + " VAL_CHAR STRING," + + " VAL_NCHAR STRING," + + " VAL_BF FLOAT," + + " VAL_BD DOUBLE," + + " VAL_F FLOAT," + + " VAL_F_10 FLOAT," + + " VAL_NUM DECIMAL(10, 6)," + + " VAL_DP DOUBLE," + + " VAL_R DECIMAL(38,2)," + + " VAL_DECIMAL DECIMAL(10, 6)," + + " VAL_NUMERIC DECIMAL(10, 6)," + + " VAL_NUM_VS DECIMAL(10, 3)," + + " VAL_INT DECIMAL(38,0)," + + " VAL_INTEGER DECIMAL(38,0)," + + " VAL_SMALLINT DECIMAL(38,0)," + + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0)," + + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0)," + + " VAL_NUMBER_1 BOOLEAN," + + " VAL_NUMBER_2 TINYINT," + + " VAL_NUMBER_4 SMALLINT," + + " VAL_NUMBER_9 INT," + + " VAL_NUMBER_18 BIGINT," + + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT," + + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT," + + " VAL_NUMBER_9_NEGATIVE_SCALE INT," + + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT," + + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0)," + + " VAL_DATE TIMESTAMP," + + " VAL_TS TIMESTAMP," + + " VAL_TS_PRECISION2 TIMESTAMP(2)," + + " VAL_TS_PRECISION4 TIMESTAMP(4)," + + " VAL_TS_PRECISION9 TIMESTAMP(6)," + + " VAL_CLOB_INLINE STRING," + + " VAL_BLOB_INLINE BYTES," + + " PRIMARY KEY (ID) NOT ENFORCED" + + ") WITH (" + + commonOptionString() + + " 'table-list' = '%s'" + + ")", + schema + ".FULL_TYPES"); + + String sinkDDL = + "CREATE TABLE sink (" + + " ID INT," + + " VAL_VARCHAR STRING," + + " VAL_VARCHAR2 STRING," + + " VAL_NVARCHAR2 STRING," + + " VAL_CHAR STRING," + + " VAL_NCHAR STRING," + + " VAL_BF FLOAT," + + " VAL_BD DOUBLE," + + " VAL_F FLOAT," + + " VAL_F_10 FLOAT," + + " VAL_NUM DECIMAL(10, 6)," + + " VAL_DP DOUBLE," + + " VAL_R DECIMAL(38,2)," + + " VAL_DECIMAL DECIMAL(10, 6)," + + " VAL_NUMERIC DECIMAL(10, 6)," + + " VAL_NUM_VS DECIMAL(10, 3)," + + " VAL_INT DECIMAL(38,0)," + + " VAL_INTEGER DECIMAL(38,0)," + + " VAL_SMALLINT DECIMAL(38,0)," + + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0)," + + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0)," + + " VAL_NUMBER_1 BOOLEAN," + + " VAL_NUMBER_2 TINYINT," + + " VAL_NUMBER_4 SMALLINT," + + " VAL_NUMBER_9 INT," + + " VAL_NUMBER_18 BIGINT," + + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT," + + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT," + + " VAL_NUMBER_9_NEGATIVE_SCALE INT," + + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT," + + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0)," + + " VAL_DATE TIMESTAMP," + + " VAL_TS TIMESTAMP," + + " VAL_TS_PRECISION2 TIMESTAMP(2)," + + " VAL_TS_PRECISION4 TIMESTAMP(4)," + + " VAL_TS_PRECISION9 TIMESTAMP(6)," + + " VAL_CLOB_INLINE STRING," + + " VAL_BLOB_INLINE STRING," + + " PRIMARY KEY (ID) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'," + + " 'sink-expected-messages-num' = '2'" + + ")"; + + tEnv.executeSql(sourceDDL); + tEnv.executeSql(sinkDDL); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink SELECT " + + " ID," + + " VAL_VARCHAR," + + " VAL_VARCHAR2," + + " VAL_NVARCHAR2," + + " VAL_CHAR," + + " VAL_NCHAR," + + " VAL_BF," + + " VAL_BD," + + " VAL_F," + + " VAL_F_10," + + " VAL_NUM," + + " VAL_DP," + + " VAL_R," + + " VAL_DECIMAL," + + " VAL_NUMERIC," + + " VAL_NUM_VS," + + " VAL_INT," + + " VAL_INTEGER," + + " VAL_SMALLINT," + + " VAL_NUMBER_38_NO_SCALE," + + " VAL_NUMBER_38_SCALE_0," + + " VAL_NUMBER_1," + + " VAL_NUMBER_2," + + " VAL_NUMBER_4," + + " VAL_NUMBER_9," + + " VAL_NUMBER_18," + + " VAL_NUMBER_2_NEGATIVE_SCALE," + + " VAL_NUMBER_4_NEGATIVE_SCALE," + + " VAL_NUMBER_9_NEGATIVE_SCALE," + + " VAL_NUMBER_18_NEGATIVE_SCALE," + + " VAL_NUMBER_36_NEGATIVE_SCALE," + + " VAL_DATE," + + " VAL_TS," + + " VAL_TS_PRECISION2," + + " VAL_TS_PRECISION4," + + " VAL_TS_PRECISION9," + + " VAL_CLOB_INLINE," + + " DECODE(VAL_BLOB_INLINE, 'UTF-8')" + + " FROM full_types"); + + waitForSinkSize("sink", 1); + + try (Connection connection = getJdbcConnection(); + Statement statement = connection.createStatement()) { + statement.execute( + "UPDATE FULL_TYPES SET VAL_TS = '2022-10-30 12:34:56.12545' WHERE id=1;"); + } + + waitForSinkSize("sink", 2); + + List expected = + Arrays.asList( + "+I(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.007890,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)", + "+U(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.125450,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)"); + + List actual = TestValuesTableFactory.getRawResults("sink"); + assertContainsInAnyOrder(expected, actual); + result.getJobClient().get().cancel().get(); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java index f759a902bff..9861df5dce3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.oceanbase.table; +import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; @@ -104,7 +105,7 @@ public void testCommonProperties() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA, - StartupMode.LATEST_OFFSET, + StartupOptions.latest(), USERNAME, PASSWORD, TENANT_NAME, @@ -113,8 +114,8 @@ public void testCommonProperties() { TABLE_LIST, SERVER_TIME_ZONE, Duration.parse("PT" + CONNECT_TIMEOUT), - null, - null, + HOSTNAME, + PORT, COMPATIBLE_MODE, DRIVER_CLASS, new Properties(), @@ -125,6 +126,7 @@ public void testCommonProperties() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); assertEquals(expectedSource, actualSource); } @@ -136,8 +138,6 @@ public void testOptionalProperties() { options.put("database-name", DATABASE_NAME); options.put("table-name", TABLE_NAME); options.put("table-list", TABLE_LIST); - options.put("hostname", HOSTNAME); - options.put("port", String.valueOf(PORT)); options.put("compatible-mode", COMPATIBLE_MODE); options.put("jdbc.driver", DRIVER_CLASS); options.put("logproxy.client.id", LOG_PROXY_CLIENT_ID); @@ -147,7 +147,7 @@ public void testOptionalProperties() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA, - StartupMode.INITIAL, + StartupOptions.initial(), USERNAME, PASSWORD, TENANT_NAME, @@ -168,6 +168,7 @@ public void testOptionalProperties() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); assertEquals(expectedSource, actualSource); } @@ -190,7 +191,7 @@ public void testMetadataColumns() { OceanBaseTableSource expectedSource = new OceanBaseTableSource( SCHEMA_WITH_METADATA, - StartupMode.LATEST_OFFSET, + StartupOptions.latest(), USERNAME, PASSWORD, TENANT_NAME, @@ -199,8 +200,8 @@ public void testMetadataColumns() { TABLE_LIST, SERVER_TIME_ZONE, Duration.parse("PT" + CONNECT_TIMEOUT), - null, - null, + HOSTNAME, + PORT, COMPATIBLE_MODE, DRIVER_CLASS, new Properties(), @@ -211,6 +212,7 @@ public void testMetadataColumns() { RS_LIST, null, WORKING_MODE, + new Properties(), new Properties()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = @@ -240,6 +242,8 @@ private Map getRequiredOptions() { options.put("scan.startup.mode", STARTUP_MODE); options.put("username", USERNAME); options.put("password", PASSWORD); + options.put("hostname", HOSTNAME); + options.put("port", String.valueOf(PORT)); options.put("tenant-name", TENANT_NAME); options.put("logproxy.host", LOG_PROXY_HOST); options.put("logproxy.port", String.valueOf(LOG_PROXY_PORT)); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql deleted file mode 100644 index 85b40fa8010..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory_meta.sql +++ /dev/null @@ -1,42 +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. - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: inventory_meta --- ---------------------------------------------------------------------------------------------------------------- - -CREATE DATABASE inventory_meta; -USE inventory_meta; - --- Create and populate our products using a single insert with many rows -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) -); -ALTER TABLE products AUTO_INCREMENT = 101; - -INSERT INTO products -VALUES (default, "scooter", "Small 2-wheel scooter", 3.14), - (default, "car battery", "12V car battery", 8.1), - (default, "12-pack drill bits", "12-pack of drill bits with sizes ranging from #40 to #3", 0.8), - (default, "hammer", "12oz carpenter's hammer", 0.75), - (default, "hammer", "14oz carpenter's hammer", 0.875), - (default, "hammer", "16oz carpenter's hammer", 1.0), - (default, "rocks", "box of assorted rocks", 5.3), - (default, "jacket", "water resistent black wind breaker", 0.1), - (default, "spare tire", "24 inch spare tire", 22.2); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql similarity index 97% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql index 98ac5255cc0..2840c9f83de 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/column_type_test.sql +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql @@ -17,9 +17,10 @@ -- DATABASE: column_type_test -- ---------------------------------------------------------------------------------------------------------------- -CREATE DATABASE column_type_test; +CREATE DATABASE IF NOT EXISTS column_type_test; USE column_type_test; +DROP TABLE IF EXISTS full_types; CREATE TABLE full_types ( id INT AUTO_INCREMENT NOT NULL, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql new file mode 100644 index 00000000000..0db9c71db4a --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/docker_init.sql @@ -0,0 +1,17 @@ +-- 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. + +-- Set the root user password of test tenant +ALTER USER root IDENTIFIED BY '123456'; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql similarity index 95% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql index bb977ad7158..2252ee19573 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/inventory.sql +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql @@ -17,10 +17,10 @@ -- DATABASE: inventory -- ---------------------------------------------------------------------------------------------------------------- -CREATE DATABASE inventory; +CREATE DATABASE IF NOT EXISTS inventory; USE inventory; --- Create and populate our products using a single insert with many rows +DROP TABLE IF EXISTS products; CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql new file mode 100644 index 00000000000..bfe622cfb10 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql @@ -0,0 +1,70 @@ +-- 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 FULL_TYPES ( + ID NUMBER(9) NOT NULL, + VAL_VARCHAR VARCHAR2(1000), + VAL_VARCHAR2 VARCHAR2(1000), + VAL_NVARCHAR2 NVARCHAR2(1000), + VAL_CHAR CHAR(3), + VAL_NCHAR NCHAR(3), + VAL_BF BINARY_FLOAT, + VAL_BD BINARY_DOUBLE, + VAL_F FLOAT, + VAL_F_10 FLOAT(10), + VAL_NUM NUMBER(10, 6), + VAL_DP FLOAT, + VAL_R FLOAT(63), + VAL_DECIMAL NUMBER(10, 6), + VAL_NUMERIC NUMBER(10, 6), + VAL_NUM_VS NUMBER, + VAL_INT NUMBER, + VAL_INTEGER NUMBER, + VAL_SMALLINT NUMBER, + VAL_NUMBER_38_NO_SCALE NUMBER(38), + VAL_NUMBER_38_SCALE_0 NUMBER(38), + VAL_NUMBER_1 NUMBER(1), + VAL_NUMBER_2 NUMBER(2), + VAL_NUMBER_4 NUMBER(4), + VAL_NUMBER_9 NUMBER(9), + VAL_NUMBER_18 NUMBER(18), + VAL_NUMBER_2_NEGATIVE_SCALE NUMBER(1, -1), + VAL_NUMBER_4_NEGATIVE_SCALE NUMBER(2, -2), + VAL_NUMBER_9_NEGATIVE_SCALE NUMBER(8, -1), + VAL_NUMBER_18_NEGATIVE_SCALE NUMBER(16, -2), + VAL_NUMBER_36_NEGATIVE_SCALE NUMBER(36, -2), + VAL_DATE DATE, + VAL_TS TIMESTAMP(6), + VAL_TS_PRECISION2 TIMESTAMP(2), + VAL_TS_PRECISION4 TIMESTAMP(4), + VAL_TS_PRECISION9 TIMESTAMP(6), + VAL_CLOB_INLINE CLOB, + VAL_BLOB_INLINE BLOB, + primary key (ID) +); + +INSERT INTO FULL_TYPES VALUES ( + 1, 'vc2', 'vc2', 'nvc2', 'c', 'nc', + 1.1, 2.22, 3.33, 8.888, 4.4444, 5.555, 6.66, 1234.567891, 1234.567891, 77.323, + 1, 22, 333, 4444, 5555, 1, 99, 9999, 999999999, 999999999999999999, + 94, 9949, 999999994, 999999999999999949, 99999999999999999999999999999999999949, + TO_DATE('2022-10-30', 'yyyy-mm-dd'), + TO_TIMESTAMP('2022-10-30 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'), + TO_TIMESTAMP('2022-10-30 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'), + TO_CLOB ('col_clob'), + utl_raw.cast_to_raw ('col_blob') +);