From 63da373f24b5617e80ab81bdde4bdac85b452243 Mon Sep 17 00:00:00 2001 From: Wenjun Ruan Date: Fri, 22 Sep 2023 16:49:35 +0800 Subject: [PATCH] Support config column/primaryKey/constraintKey in schema --- docs/en/concept/schema-feature.md | 154 ++++++- docs/en/connector-v2/sink/Assert.md | 79 +++- .../api/configuration/ReadonlyConfig.java | 19 +- .../api/configuration/util/ConfigUtil.java | 9 +- .../api/table/catalog/CatalogTableUtil.java | 210 +--------- .../SeaTunnelDataTypeConvertorUtil.java | 173 ++++++++ .../catalog/schema/ReadonlyConfigParser.java | 393 ++++++++++++++++++ .../catalog/schema/TableSchemaOptions.java | 155 +++++++ .../catalog/schema/TableSchemaParser.java | 81 ++++ .../catalog/schema/BaseConfigParserTest.java | 51 +++ .../schema/ReadonlyConfigParserTest.java | 112 +++++ .../resources/conf/catalog/schema_column.conf | 189 +++++++++ .../resources/conf/catalog/schema_field.conf | 93 +++++ .../common/config/TypesafeConfigUtils.java | 5 +- .../seatunnel/common/utils/JsonUtils.java | 18 + .../config/AmazonDynamoDBSourceOptions.java | 6 +- .../source/AmazonDynamoDBSource.java | 4 +- .../source/AmazonDynamoDBSourceFactory.java | 4 +- .../exception/AssertConnectorErrorCode.java | 4 +- .../rule/AssertCatalogTableRule.java | 141 +++++++ .../rule/AssertCatalogTableRuleParser.java | 159 +++++++ .../assertion/rule/AssertRuleParser.java | 4 + .../assertion/sink/AssertConfig.java | 36 +- .../seatunnel/assertion/sink/AssertSink.java | 76 +++- .../assertion/sink/AssertSinkFactory.java | 7 + .../seatunnel/assertion/sink/Rules.java | 4 + .../source/ElasticsearchSource.java | 3 +- .../source/ElasticsearchSourceFactory.java | 4 +- .../seatunnel/fake/source/FakeSource.java | 34 +- .../fake/source/FakeSourceFactory.java | 16 +- .../file/hdfs/source/BaseHdfsFileSource.java | 3 +- .../file/cos/source/CosFileSource.java | 3 +- .../file/cos/source/CosFileSourceFactory.java | 4 +- .../file/ftp/source/FtpFileSource.java | 3 +- .../file/ftp/source/FtpFileSourceFactory.java | 4 +- .../hdfs/source/HdfsFileSourceFactory.java | 4 +- .../file/oss/source/OssFileSource.java | 3 +- .../file/oss/source/OssFileSourceFactory.java | 4 +- .../file/local/source/LocalFileSource.java | 3 +- .../local/source/LocalFileSourceFactory.java | 4 +- .../file/oss/source/OssFileSource.java | 3 +- .../file/oss/source/OssFileSourceFactory.java | 4 +- .../file/s3/source/S3FileSource.java | 3 +- .../file/s3/source/S3FileSourceFactory.java | 4 +- .../file/sftp/source/SftpFileSource.java | 3 +- .../sftp/source/SftpFileSourceFactory.java | 4 +- .../google/sheets/source/SheetsSource.java | 5 +- .../sheets/source/SheetsSourceFactory.java | 4 +- .../seatunnel/hive/source/HiveSource.java | 8 +- .../seatunnel/http/source/HttpSource.java | 3 +- .../http/source/HttpSourceFactory.java | 6 +- .../iceberg/source/IcebergSource.java | 3 +- .../iceberg/source/IcebergSourceFactory.java | 4 +- .../influxdb/source/InfluxDBSource.java | 3 +- .../source/InfluxDBSourceFactory.java | 4 +- .../seatunnel/iotdb/source/IoTDBSource.java | 3 +- .../iotdb/source/IoTDBSourceFactory.java | 4 +- .../mongodb/sink/MongodbSinkFactory.java | 4 +- .../mongodb/source/MongodbSource.java | 3 +- .../mongodb/source/MongodbSourceFactory.java | 4 +- .../seatunnel/neo4j/source/Neo4jSource.java | 3 +- .../neo4j/source/Neo4jSourceFactory.java | 4 +- .../pulsar/source/PulsarSourceFactory.java | 4 +- .../source/format/PulsarCanalDecorator.java | 2 +- .../rabbitmq/source/RabbitmqSource.java | 4 +- .../source/RabbitmqSourceFactory.java | 4 +- .../seatunnel/redis/source/RedisSource.java | 3 +- .../redis/source/RedisSourceFactory.java | 4 +- .../starrocks/source/StarRocksSource.java | 3 +- .../source/StarRocksSourceFactory.java | 5 +- .../sink/TablestoreSinkFactory.java | 4 +- .../e2e/connector/fake/FakeWithSchemaTT.java | 42 ++ .../fake_to_assert_with_catalogtable.conf | 99 +++++ .../seatunnel/engine/server/TestUtils.java | 5 +- .../server/checkpoint/CheckpointPlanTest.java | 3 +- .../seatunnel/engine/server/dag/TaskTest.java | 3 +- 76 files changed, 2214 insertions(+), 335 deletions(-) create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/SeaTunnelDataTypeConvertorUtil.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaOptions.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaParser.java create mode 100644 seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/BaseConfigParserTest.java create mode 100644 seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParserTest.java create mode 100644 seatunnel-api/src/test/resources/conf/catalog/schema_column.conf create mode 100644 seatunnel-api/src/test/resources/conf/catalog/schema_field.conf create mode 100644 seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRule.java create mode 100644 seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRuleParser.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/resources/fake_to_assert_with_catalogtable.conf diff --git a/docs/en/concept/schema-feature.md b/docs/en/concept/schema-feature.md index ebc0cf68a461..61714e6c9e8c 100644 --- a/docs/en/concept/schema-feature.md +++ b/docs/en/concept/schema-feature.md @@ -2,9 +2,55 @@ ## Why we need schema -Some NoSQL databases or message queue are not strongly limited schema, so the schema cannot be obtained through the api. At this time, a schema needs to be defined to convert to SeaTunnelRowType and obtain data. +Some NoSQL databases or message queue are not strongly limited schema, so the schema cannot be obtained through the api. +At this time, a schema needs to be defined to convert to TableSchema and obtain data. -## What type supported at now +## SchemaOptions + +We can use SchemaOptions to define schema, the SchemaOptions contains some config to define the schema. e.g. columns, primaryKey, constraintKeys. + +``` +schema = { + columns = [ + ... + ] + primaryKey { + ... + } + + constraintKeys { + ... + } +} +``` + +### Columns + +Columns is a list of config used to define the column in schema, each column can contains name, type, nullable, defaultValue, comment field. + +``` +columns = [ + { + name = id + type = bigint + nullable = false + columnLength = 20 + defaultValue = 0 + comment = "primary key id" + } +] +``` + +| Field | Required | Default Value | Description | +|:-------------|:---------|:--------------|----------------------------------------------------------------------------------| +| name | Yes | - | The name of the column | +| type | Yes | - | The data type of the column | +| nullable | No | true | If the column can be nullable | +| columnLength | No | 0 | The length of the column which will be useful when you need to define the length | +| defaultValue | No | null | The default value of the column | +| comment | No | null | The comment of the column | + +#### What type supported at now | Data type | Value type in Java | Description | |:----------|:---------------------------------------------------|:------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| @@ -26,9 +72,111 @@ Some NoSQL databases or message queue are not strongly limited schema, so the sc | map | `java.util.Map` | A Map is an object that maps keys to values. The key type includes `int` `string` `boolean` `tinyint` `smallint` `bigint` `float` `double` `decimal` `date` `time` `timestamp` `null` , and the value type includes `int` `string` `boolean` `tinyint` `smallint` `bigint` `float` `double` `decimal` `date` `time` `timestamp` `null` `array` `map`. | | array | `ValueType[]` | A array is a data type that represents a collection of elements. The element type includes `int` `string` `boolean` `tinyint` `smallint` `bigint` `float` `double` `array` `map`. | +### PrimaryKey + +Primary key is a config used to define the primary key in schema, it contains name, columns field. + +``` +primaryKey { + name = id + columns = [id] +} +``` + +| Field | Required | Default Value | Description | +|:--------|:---------|:--------------|-----------------------------------| +| name | Yes | - | The name of the primaryKey | +| columns | Yes | - | The column list in the primaryKey | + +### ConstraintKeys + +Constraint keys is a list of config used to define the constraint keys in schema, it contains constraintName, constraintType, constraintColumns field. + +``` +constraintKeys = [ + { + constraintName = "id_index" + constraintType = KEY + constraintColumns = [ + { + columnName = "id" + sortType = ASC + } + ] + }, + ] +``` + +| Field | Required | Default Value | Description | +|:------------------|:---------|:--------------|-------------------------------------------------------------------------------------------------------------------------------------------| +| constraintName | Yes | - | The name of the constraintKey | +| constraintType | No | KEY | The type of the constraintKey | +| constraintColumns | Yes | - | The column list in the primaryKey, each column should contains constraintType and sortType, sortType support ASC and DESC, default is ASC | + +#### What constraintType supported at now + +| ConstraintType | Description | +|:---------------|:------------| +| KEY | key | +| UNIQUE_KEY | unique key | +| FOREIGN_KEY | foreign key | + ## How to use schema -`schema` defines the format of the data,it contains`fields` properties. `fields` define the field properties,it's a K-V key-value pair, the Key is the field name and the value is field type. Here is an example. +### Recommended + +``` +source { + FakeSource { + parallelism = 2 + result_table_name = "fake" + row.num = 16 + schema { + columns = [ + { + name = id + type = bigint + nullable = false + defaultValue = 0 + comment = "primary key id" + }, + { + name = name + type = "string" + nullable = true + comment = "name" + }, + { + name = age + type = int + nullable = true + comment = "age" + } + ] + primaryKey { + name = "id" + columnNames = [id] + } + constraintKeys = [ + { + constraintName = "unique_name" + constraintType = UNIQUE_KEY + constraintColumns = [ + { + columnName = "name" + sortType = ASC + } + ] + }, + ] + } + } +} +``` + +### Deprecated + +If you only need to define the column, you can use fields to define the column, this is a simple way but will be remove in the future. ``` source { diff --git a/docs/en/connector-v2/sink/Assert.md b/docs/en/connector-v2/sink/Assert.md index f924e3d2eb5a..175df9285406 100644 --- a/docs/en/connector-v2/sink/Assert.md +++ b/docs/en/connector-v2/sink/Assert.md @@ -12,19 +12,36 @@ A flink sink plugin which can assert illegal data by user defined rules ## Options -| name | type | required | default value | -|------------------------------------------|------------|----------|---------------| -| rules | ConfigMap | yes | - | -| rules.field_rules | string | yes | - | -| rules.field_rules.field_name | string | yes | - | -| rules.field_rules.field_type | string | no | - | -| rules.field_rules.field_value | ConfigList | no | - | -| rules.field_rules.field_value.rule_type | string | no | - | -| rules.field_rules.field_value.rule_value | double | no | - | -| rules.row_rules | string | yes | - | -| rules.row_rules.rule_type | string | no | - | -| rules.row_rules.rule_value | string | no | - | -| common-options | | no | - | +| name | type | required | default value | +|------------------------------------------------------------------------------------------------|------------|----------|---------------| +| rules | ConfigMap | yes | - | +| rules.field_rules | string | yes | - | +| rules.field_rules.field_name | string | yes | - | +| rules.field_rules.field_type | string | no | - | +| rules.field_rules.field_value | ConfigList | no | - | +| rules.field_rules.field_value.rule_type | string | no | - | +| rules.field_rules.field_value.rule_value | double | no | - | +| rules.row_rules | string | yes | - | +| rules.row_rules.rule_type | string | no | - | +| rules.row_rules.rule_value | string | no | - | +| rules.catalog_table_rule | ConfigMap | no | - | +| rules.catalog_table_rule.primary_key_rule | ConfigMap | no | - | +| rules.catalog_table_rule.primary_key_rule.primary_key_name | string | no | - | +| rules.catalog_table_rule.primary_key_rule.primary_key_columns | list | no | - | +| rules.catalog_table_rule.constraint_key_rule | ConfigList | no | - | +| rules.catalog_table_rule.constraint_key_rule.constraint_key_name | string | no | - | +| rules.catalog_table_rule.constraint_key_rule.constraint_key_type | string | no | - | +| rules.catalog_table_rule.constraint_key_rule.constraint_key_columns | ConfigList | no | - | +| rules.catalog_table_rule.constraint_key_rule.constraint_key_columns.constraint_key_column_name | string | no | - | +| rules.catalog_table_rule.constraint_key_rule.constraint_key_columns.constraint_key_sort_type | string | no | - | +| rules.catalog_table_rule.column_rule | ConfigList | no | - | +| rules.catalog_table_rule.column_rule.name | string | no | - | +| rules.catalog_table_rule.column_rule.type | string | no | - | +| rules.catalog_table_rule.column_rule.column_length | int | no | - | +| rules.catalog_table_rule.column_rule.nullable | boolean | no | - | +| rules.catalog_table_rule.column_rule.default_value | string | no | - | +| rules.catalog_table_rule.column_rule.comment | comment | no | - | +| common-options | | no | - | ### rules [ConfigMap] @@ -61,6 +78,10 @@ The following rules are supported for now the value related to rule type +### catalog_table_rule [ConfigMap] + +Used to assert the catalog table is same with the user defined table. + ### common options Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details @@ -117,6 +138,38 @@ Assert { ] } ] + catalog_table_rule { + primary_key_rule = { + primary_key_name = "primary key" + primary_key_columns = ["id"] + } + constraint_key_rule = [ + { + constraint_key_name = "unique_name" + constraint_key_type = UNIQUE_KEY + constraint_key_columns = [ + { + constraint_key_column_name = "id" + constraint_key_sort_type = ASC + } + ] + } + ] + column_rule = [ + { + name = "id" + type = bigint + }, + { + name = "name" + type = string + }, + { + name = "age" + type = int + } + ] + } } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java index 808109ea5cf3..b3db36245445 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java @@ -72,21 +72,34 @@ public T get(Option option, boolean flatten) { } public Map toMap() { + return toMap(true); + } + + public Map toMap(boolean flatten) { if (confData.isEmpty()) { return Collections.emptyMap(); } Map result = new LinkedHashMap<>(); - toMap(result); + toMap(result, flatten); return result; } public void toMap(Map result) { + toMap(result, true); + } + + public void toMap(Map result, boolean flatten) { if (confData.isEmpty()) { return; } - Map flatteningMap = flatteningMap(confData); - for (Map.Entry entry : flatteningMap.entrySet()) { + Map map; + if (flatten) { + map = flatteningMap(confData); + } else { + map = confData; + } + for (Map.Entry entry : map.entrySet()) { result.put(entry.getKey(), convertToJsonString(entry.getValue())); } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/util/ConfigUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/util/ConfigUtil.java index 097d6d10ae39..4498928c93e2 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/util/ConfigUtil.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/util/ConfigUtil.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.commons.lang3.StringEscapeUtils; import org.apache.commons.lang3.StringUtils; @@ -41,8 +42,6 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; - @Slf4j public class ConfigUtil { private static final JavaPropsMapper PROPERTIES_MAPPER = new JavaPropsMapper(); @@ -72,8 +71,10 @@ public static Map treeMap(Map rawMap) { Map result = loadPropertiesStyleMap(properties); // Special case, we shouldn't change key in schema config. // TODO we should not hard code it, it should be as a config. - if (rawMap.containsKey(SCHEMA.key())) { - result.put(SCHEMA.key(), rawMap.get(SCHEMA.key())); + if (rawMap.containsKey(TableSchemaOptions.SCHEMA.key())) { + result.put( + TableSchemaOptions.SCHEMA.key(), + rawMap.get(TableSchemaOptions.SCHEMA.key())); } return result; } catch (JsonProcessingException e) { diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java index def005eeb6b0..557759ef3eb8 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java @@ -17,29 +17,18 @@ package org.apache.seatunnel.api.table.catalog; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.JsonNodeType; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.schema.ReadonlyConfigParser; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.FactoryUtil; -import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.DecimalType; -import org.apache.seatunnel.api.table.type.LocalTimeType; -import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.MultipleRowType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.api.table.type.SqlType; -import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; import lombok.extern.slf4j.Slf4j; @@ -48,21 +37,13 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; +/** Utils contains some common methods for construct CatalogTable. */ @Slf4j public class CatalogTableUtil implements Serializable { - public static final Option> SCHEMA = - Options.key("schema").mapType().noDefaultValue().withDescription("SeaTunnel Schema"); - - public static final Option> FIELDS = - Options.key("schema.fields") - .type(new TypeReference>() {}) - .noDefaultValue() - .withDescription("SeaTunnel Schema Fields"); private static final SeaTunnelRowType SIMPLE_SCHEMA = new SeaTunnelRowType( @@ -88,6 +69,12 @@ public static CatalogTable getCatalogTable(String tableName, SeaTunnelRowType ro // TODO remove this method after https://github.com/apache/seatunnel/issues/5483 done. @Deprecated public static List getCatalogTables(Config config, ClassLoader classLoader) { + // Highest priority: specified schema + if (config.hasPath(TableSchemaOptions.SCHEMA.key())) { + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + return Collections.singletonList(catalogTable); + } + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(config); Map catalogOptions = readonlyConfig.getOptional(CatalogOptions.CATALOG_OPTIONS).orElse(new HashMap<>()); @@ -96,20 +83,13 @@ public static List getCatalogTables(Config config, ClassLoader cla catalogAllOptions.putAll(readonlyConfig.toMap()); catalogAllOptions.putAll(catalogOptions); ReadonlyConfig catalogConfig = ReadonlyConfig.fromMap(catalogAllOptions); - String factoryId = catalogConfig.get(CommonOptions.FACTORY_ID); - // Highest priority: specified schema - Map schemaMap = readonlyConfig.get(CatalogTableUtil.SCHEMA); - if (schemaMap != null && schemaMap.size() > 0) { - CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); - return Collections.singletonList(catalogTable); - } Optional optionalCatalog = FactoryUtil.createOptionalCatalog( catalogConfig.get(CatalogOptions.NAME), catalogConfig, classLoader, - factoryId); + catalogConfig.get(CommonOptions.FACTORY_ID)); return optionalCatalog .map( c -> { @@ -119,7 +99,7 @@ public static List getCatalogTables(Config config, ClassLoader cla List catalogTables = catalog.getTables(catalogConfig); log.info( String.format( - "Get catalog tables, cost time: %d", + "Get catalog tables, cost time: %d ms", System.currentTimeMillis() - startTime)); return catalogTables; } @@ -140,6 +120,7 @@ public static List getCatalogTables(Config config, ClassLoader cla @Deprecated public static List getCatalogTablesFromConfig( ReadonlyConfig readonlyConfig, ClassLoader classLoader) { + // We use plugin_name as factoryId, so MySQL-CDC should be MySQL String factoryId = readonlyConfig.get(CommonOptions.PLUGIN_NAME).replace("-CDC", ""); return getCatalogTablesFromConfig(factoryId, readonlyConfig, classLoader); @@ -149,7 +130,7 @@ public static List getCatalogTablesFromConfig( public static List getCatalogTablesFromConfig( String factoryId, ReadonlyConfig readonlyConfig, ClassLoader classLoader) { // Highest priority: specified schema - Map schemaMap = readonlyConfig.get(CatalogTableUtil.SCHEMA); + Map schemaMap = readonlyConfig.get(TableSchemaOptions.SCHEMA); if (schemaMap != null) { if (schemaMap.isEmpty()) { throw new SeaTunnelException("Schema config can not be empty"); @@ -210,11 +191,11 @@ public static SeaTunnelDataType convertToDataType( } public static CatalogTable buildWithConfig(ReadonlyConfig readonlyConfig) { - if (readonlyConfig.get(SCHEMA) == null) { + if (readonlyConfig.get(TableSchemaOptions.SCHEMA) == null) { throw new RuntimeException( "Schema config need option [schema], please correct your config first"); } - TableSchema tableSchema = parseTableSchema(readonlyConfig); + TableSchema tableSchema = new ReadonlyConfigParser().parse(readonlyConfig); return CatalogTable.of( // TODO: other table info TableIdentifier.of("", "", ""), @@ -227,165 +208,4 @@ public static CatalogTable buildWithConfig(ReadonlyConfig readonlyConfig) { public static SeaTunnelRowType buildSimpleTextSchema() { return SIMPLE_SCHEMA; } - - public static SeaTunnelDataType parseDataType(String columnStr) { - SqlType sqlType = null; - try { - sqlType = SqlType.valueOf(columnStr.toUpperCase().replace(" ", "")); - } catch (IllegalArgumentException e) { - // nothing - } - if (sqlType == null) { - return parseComplexDataType(columnStr); - } - switch (sqlType) { - case STRING: - return BasicType.STRING_TYPE; - case BOOLEAN: - return BasicType.BOOLEAN_TYPE; - case TINYINT: - return BasicType.BYTE_TYPE; - case BYTES: - return PrimitiveByteArrayType.INSTANCE; - case SMALLINT: - return BasicType.SHORT_TYPE; - case INT: - return BasicType.INT_TYPE; - case BIGINT: - return BasicType.LONG_TYPE; - case FLOAT: - return BasicType.FLOAT_TYPE; - case DOUBLE: - return BasicType.DOUBLE_TYPE; - case NULL: - return BasicType.VOID_TYPE; - case DATE: - return LocalTimeType.LOCAL_DATE_TYPE; - case TIME: - return LocalTimeType.LOCAL_TIME_TYPE; - case TIMESTAMP: - return LocalTimeType.LOCAL_DATE_TIME_TYPE; - default: - throw new UnsupportedOperationException( - String.format("the type[%s] is not support", columnStr)); - } - } - - private static SeaTunnelDataType parseComplexDataType(String columnStr) { - String column = columnStr.toUpperCase().replace(" ", ""); - if (column.startsWith(SqlType.MAP.name())) { - return parseMapType(column); - } - if (column.startsWith(SqlType.ARRAY.name())) { - return parseArrayType(column); - } - if (column.startsWith(SqlType.DECIMAL.name())) { - return parseDecimalType(column); - } - return parseRowType(columnStr); - } - - private static SeaTunnelDataType parseRowType(String columnStr) { - Map fieldsMap = convertJsonToMap(JsonUtils.parseObject(columnStr)); - String[] fieldsName = new String[fieldsMap.size()]; - SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[fieldsMap.size()]; - int i = 0; - for (Map.Entry entry : fieldsMap.entrySet()) { - fieldsName[i] = entry.getKey(); - seaTunnelDataTypes[i] = parseDataType(entry.getValue()); - i++; - } - return new SeaTunnelRowType(fieldsName, seaTunnelDataTypes); - } - - private static SeaTunnelDataType parseMapType(String columnStr) { - String genericType = getGenericType(columnStr); - int index = - genericType.startsWith(SqlType.DECIMAL.name()) - ? - // if map key is decimal, we should find the index of second ',' - genericType.indexOf(",", genericType.indexOf(",") + 1) - : - // if map key is not decimal, we should find the index of first ',' - genericType.indexOf(","); - String keyGenericType = genericType.substring(0, index); - String valueGenericType = genericType.substring(index + 1); - return new MapType<>(parseDataType(keyGenericType), parseDataType(valueGenericType)); - } - - private static String getGenericType(String columnStr) { - // get the content between '<' and '>' - return columnStr.substring(columnStr.indexOf("<") + 1, columnStr.lastIndexOf(">")); - } - - private static SeaTunnelDataType parseArrayType(String columnStr) { - String genericType = getGenericType(columnStr); - SeaTunnelDataType dataType = parseDataType(genericType); - switch (dataType.getSqlType()) { - case STRING: - return ArrayType.STRING_ARRAY_TYPE; - case BOOLEAN: - return ArrayType.BOOLEAN_ARRAY_TYPE; - case TINYINT: - return ArrayType.BYTE_ARRAY_TYPE; - case SMALLINT: - return ArrayType.SHORT_ARRAY_TYPE; - case INT: - return ArrayType.INT_ARRAY_TYPE; - case BIGINT: - return ArrayType.LONG_ARRAY_TYPE; - case FLOAT: - return ArrayType.FLOAT_ARRAY_TYPE; - case DOUBLE: - return ArrayType.DOUBLE_ARRAY_TYPE; - default: - String errorMsg = - String.format("Array type not support this genericType [%s]", genericType); - throw new UnsupportedOperationException(errorMsg); - } - } - - private static SeaTunnelDataType parseDecimalType(String columnStr) { - String[] decimalInfos = columnStr.split(","); - if (decimalInfos.length < 2) { - throw new RuntimeException( - "Decimal type should assign precision and scale information"); - } - int precision = Integer.parseInt(decimalInfos[0].replaceAll("\\D", "")); - int scale = Integer.parseInt(decimalInfos[1].replaceAll("\\D", "")); - return new DecimalType(precision, scale); - } - - private static Map convertJsonToMap(ObjectNode jsonNodes) { - LinkedHashMap fieldsMap = new LinkedHashMap<>(); - jsonNodes - .fields() - .forEachRemaining( - field -> { - String key = field.getKey(); - JsonNode value = field.getValue(); - if (value.getNodeType() == JsonNodeType.OBJECT) { - fieldsMap.put(key, value.toString()); - } else { - fieldsMap.put(key, value.textValue()); - } - }); - return fieldsMap; - } - - private static TableSchema parseTableSchema(ReadonlyConfig config) { - Map fieldsMap = - convertJsonToMap((ObjectNode) JsonUtils.toJsonNode(config.get(FIELDS, false))); - int fieldsNum = fieldsMap.size(); - List columns = new ArrayList<>(fieldsNum); - for (Map.Entry entry : fieldsMap.entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - SeaTunnelDataType dataType = parseDataType(value); - // TODO: column - PhysicalColumn column = PhysicalColumn.of(key, dataType, 0, true, null, null); - columns.add(column); - } - return TableSchema.builder().columns(columns).build(); - } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/SeaTunnelDataTypeConvertorUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/SeaTunnelDataTypeConvertorUtil.java new file mode 100644 index 000000000000..272878c749b8 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/SeaTunnelDataTypeConvertorUtil.java @@ -0,0 +1,173 @@ +/* + * 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.seatunnel.api.table.catalog; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.utils.JsonUtils; + +import java.util.Map; + +public class SeaTunnelDataTypeConvertorUtil { + + /** + * @param columnType column type, should be {@link SeaTunnelDataType##toString}. + * @return {@link SeaTunnelDataType} instance. + */ + public static SeaTunnelDataType deserializeSeaTunnelDataType(String columnType) { + SqlType sqlType = null; + try { + sqlType = SqlType.valueOf(columnType.toUpperCase().replace(" ", "")); + } catch (IllegalArgumentException e) { + // nothing + } + if (sqlType == null) { + return parseComplexDataType(columnType); + } + switch (sqlType) { + case STRING: + return BasicType.STRING_TYPE; + case BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case TINYINT: + return BasicType.BYTE_TYPE; + case BYTES: + return PrimitiveByteArrayType.INSTANCE; + case SMALLINT: + return BasicType.SHORT_TYPE; + case INT: + return BasicType.INT_TYPE; + case BIGINT: + return BasicType.LONG_TYPE; + case FLOAT: + return BasicType.FLOAT_TYPE; + case DOUBLE: + return BasicType.DOUBLE_TYPE; + case NULL: + return BasicType.VOID_TYPE; + case DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case TIME: + return LocalTimeType.LOCAL_TIME_TYPE; + case TIMESTAMP: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case MAP: + return parseMapType(columnType); + default: + throw new UnsupportedOperationException( + String.format("the type[%s] is not support", columnType)); + } + } + + private static SeaTunnelDataType parseComplexDataType(String columnStr) { + String column = columnStr.toUpperCase().replace(" ", ""); + if (column.startsWith(SqlType.MAP.name())) { + return parseMapType(column); + } + if (column.startsWith(SqlType.ARRAY.name())) { + return parseArrayType(column); + } + if (column.startsWith(SqlType.DECIMAL.name())) { + return parseDecimalType(column); + } + return parseRowType(columnStr); + } + + private static SeaTunnelDataType parseRowType(String columnStr) { + ObjectNode jsonNodes = JsonUtils.parseObject(columnStr); + Map fieldsMap = JsonUtils.toStringMap(jsonNodes); + String[] fieldsName = new String[fieldsMap.size()]; + SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[fieldsMap.size()]; + int i = 0; + for (Map.Entry entry : fieldsMap.entrySet()) { + fieldsName[i] = entry.getKey(); + seaTunnelDataTypes[i] = deserializeSeaTunnelDataType(entry.getValue()); + i++; + } + return new SeaTunnelRowType(fieldsName, seaTunnelDataTypes); + } + + private static SeaTunnelDataType parseMapType(String columnStr) { + String genericType = getGenericType(columnStr); + int index = + genericType.startsWith(SqlType.DECIMAL.name()) + ? + // if map key is decimal, we should find the index of second ',' + genericType.indexOf(",", genericType.indexOf(",") + 1) + : + // if map key is not decimal, we should find the index of first ',' + genericType.indexOf(","); + String keyGenericType = genericType.substring(0, index); + String valueGenericType = genericType.substring(index + 1); + return new MapType<>( + deserializeSeaTunnelDataType(keyGenericType), + deserializeSeaTunnelDataType(valueGenericType)); + } + + private static String getGenericType(String columnStr) { + // get the content between '<' and '>' + return columnStr.substring(columnStr.indexOf("<") + 1, columnStr.lastIndexOf(">")); + } + + private static SeaTunnelDataType parseArrayType(String columnStr) { + String genericType = getGenericType(columnStr); + SeaTunnelDataType dataType = deserializeSeaTunnelDataType(genericType); + switch (dataType.getSqlType()) { + case STRING: + return ArrayType.STRING_ARRAY_TYPE; + case BOOLEAN: + return ArrayType.BOOLEAN_ARRAY_TYPE; + case TINYINT: + return ArrayType.BYTE_ARRAY_TYPE; + case SMALLINT: + return ArrayType.SHORT_ARRAY_TYPE; + case INT: + return ArrayType.INT_ARRAY_TYPE; + case BIGINT: + return ArrayType.LONG_ARRAY_TYPE; + case FLOAT: + return ArrayType.FLOAT_ARRAY_TYPE; + case DOUBLE: + return ArrayType.DOUBLE_ARRAY_TYPE; + default: + String errorMsg = + String.format("Array type not support this genericType [%s]", genericType); + throw new UnsupportedOperationException(errorMsg); + } + } + + private static SeaTunnelDataType parseDecimalType(String columnStr) { + String[] decimalInfos = columnStr.split(","); + if (decimalInfos.length < 2) { + throw new RuntimeException( + "Decimal type should assign precision and scale information"); + } + int precision = Integer.parseInt(decimalInfos[0].replaceAll("\\D", "")); + int scale = Integer.parseInt(decimalInfos[1].replaceAll("\\D", "")); + return new DecimalType(precision, scale); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java new file mode 100644 index 000000000000..5e9792c4affa --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java @@ -0,0 +1,393 @@ +/* + * 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.seatunnel.api.table.catalog.schema; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.SeaTunnelDataTypeConvertorUtil; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.utils.JsonUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class ReadonlyConfigParser implements TableSchemaParser { + + private final TableSchemaParser.ColumnParser columnParser = new ColumnParser(); + private final TableSchemaParser.FieldParser fieldParser = new FieldParser(); + private final TableSchemaParser.ConstraintKeyParser constraintKeyParser = + new ConstraintKeyParser(); + private final TableSchemaParser.PrimaryKeyParser primaryKeyParser = + new PrimaryKeyParser(); + + @Override + public TableSchema parse(ReadonlyConfig readonlyConfig) { + ReadonlyConfig schemaConfig = + readonlyConfig + .getOptional(TableSchemaOptions.SCHEMA) + .map(ReadonlyConfig::fromMap) + .orElseThrow( + () -> new IllegalArgumentException("Schema config can't be null")); + + if (readonlyConfig.getOptional(TableSchemaOptions.FieldOptions.FIELDS).isPresent() + && schemaConfig.getOptional(TableSchemaOptions.ColumnOptions.COLUMNS).isPresent()) { + throw new IllegalArgumentException( + "Schema config can't contains both [fields] and [columns], please correct your config first"); + } + TableSchema.Builder tableSchemaBuilder = TableSchema.builder(); + if (readonlyConfig.getOptional(TableSchemaOptions.FieldOptions.FIELDS).isPresent()) { + // we use readonlyConfig here to avoid flatten, this is used to solve the t.x.x as field + // key + tableSchemaBuilder.columns(fieldParser.parse(readonlyConfig)); + } + + if (schemaConfig.getOptional(TableSchemaOptions.ColumnOptions.COLUMNS).isPresent()) { + tableSchemaBuilder.columns(columnParser.parse(schemaConfig)); + } + if (schemaConfig + .getOptional(TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY) + .isPresent()) { + tableSchemaBuilder.primaryKey(primaryKeyParser.parse(schemaConfig)); + } + if (schemaConfig + .getOptional(TableSchemaOptions.ConstraintKeyOptions.CONSTRAINT_KEYS) + .isPresent()) { + tableSchemaBuilder.constraintKey(constraintKeyParser.parse(schemaConfig)); + } + // todo: validate schema + return tableSchemaBuilder.build(); + } + + public class FieldParser implements TableSchemaParser.FieldParser { + + @Override + public List parse(ReadonlyConfig schemaConfig) { + JsonNode jsonNode = + JsonUtils.toJsonNode( + schemaConfig.get(TableSchemaOptions.FieldOptions.FIELDS, false)); + Map fieldsMap = JsonUtils.toStringMap(jsonNode); + int fieldsNum = fieldsMap.size(); + List columns = new ArrayList<>(fieldsNum); + for (Map.Entry entry : fieldsMap.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + SeaTunnelDataType dataType = + SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType(value); + PhysicalColumn column = PhysicalColumn.of(key, dataType, 0, true, null, null); + columns.add(column); + } + return columns; + } + } + + public class ColumnParser implements TableSchemaParser.ColumnParser { + + @Override + public List parse(ReadonlyConfig schemaConfig) { + return schemaConfig.get(TableSchemaOptions.ColumnOptions.COLUMNS, false).stream() + .map(ReadonlyConfig::fromMap) + .map( + columnConfig -> { + String name = + columnConfig + .getOptional(TableSchemaOptions.ColumnOptions.NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.columns.* config need option [name], please correct your config first")); + SeaTunnelDataType seaTunnelDataType = + columnConfig + .getOptional( + TableSchemaOptions.ColumnOptions.TYPE, + false) + .map( + SeaTunnelDataTypeConvertorUtil + ::deserializeSeaTunnelDataType) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.columns.* config need option [type], please correct your config first")); + + Integer columnLength = + columnConfig.get( + TableSchemaOptions.ColumnOptions.COLUMN_LENGTH); + Boolean nullable = + columnConfig.get(TableSchemaOptions.ColumnOptions.NULLABLE); + Object defaultValue = + columnConfig.get( + TableSchemaOptions.ColumnOptions.DEFAULT_VALUE); + String comment = + columnConfig.get(TableSchemaOptions.ColumnOptions.COMMENT); + return PhysicalColumn.of( + name, + seaTunnelDataType, + columnLength, + nullable, + defaultValue, + comment); + }) + .collect(Collectors.toList()); + } + } + + public class ConstraintKeyParser + implements TableSchemaParser.ConstraintKeyParser { + + @Override + public List parse(ReadonlyConfig schemaConfig) { + return schemaConfig.get(TableSchemaOptions.ConstraintKeyOptions.CONSTRAINT_KEYS) + .stream() + .map(ReadonlyConfig::fromMap) + .map( + constraintKeyConfig -> { + String constraintName = + constraintKeyConfig + .getOptional( + TableSchemaOptions.ConstraintKeyOptions + .CONSTRAINT_KEY_NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [constraintName], please correct your config first")); + ConstraintKey.ConstraintType constraintType = + constraintKeyConfig + .getOptional( + TableSchemaOptions.ConstraintKeyOptions + .CONSTRAINT_KEY_TYPE) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [constraintType], please correct your config first")); + List columns = + constraintKeyConfig + .getOptional( + TableSchemaOptions.ConstraintKeyOptions + .CONSTRAINT_KEY_COLUMNS) + .map( + constraintColumnMapList -> { + return constraintColumnMapList.stream() + .map(ReadonlyConfig::fromMap) + .map( + constraintColumnConfig -> { + String columnName = + constraintColumnConfig + .getOptional( + TableSchemaOptions + .ConstraintKeyOptions + .CONSTRAINT_KEY_COLUMN_NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.constraintColumns.* config need option [columnName], please correct your config first")); + ConstraintKey + .ColumnSortType + columnSortType = + constraintColumnConfig + .get( + TableSchemaOptions + .ConstraintKeyOptions + .CONSTRAINT_KEY_COLUMN_SORT_TYPE); + return ConstraintKey + .ConstraintKeyColumn + .of( + columnName, + columnSortType); + }) + .collect(Collectors.toList()); + }) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [columns], please correct your config first")); + return ConstraintKey.of(constraintType, constraintName, columns); + }) + .collect(Collectors.toList()); + } + } + + public class PrimaryKeyParser implements TableSchemaParser.PrimaryKeyParser { + + @Override + public PrimaryKey parse(ReadonlyConfig schemaConfig) { + ReadonlyConfig primaryKeyConfig = + ReadonlyConfig.fromMap( + schemaConfig.get(TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY)); + String primaryKeyName = + primaryKeyConfig + .getOptional(TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "Schema config need option [primaryKey.name], please correct your config first")); + List columns = + primaryKeyConfig + .getOptional(TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_COLUMNS) + .orElseThrow( + () -> + new IllegalArgumentException( + "Schema config need option [primaryKey.columnNames], please correct your config first")); + return new PrimaryKey(primaryKeyName, columns); + } + } + + /** + * Parse columns from columns config. + * + *
+     *     columns = [
+     *      {
+     *          name = "name"
+     *          type = "string"
+     *          columnLength = 0
+     *          nullable = true
+     *          defaultValue = null
+     *          comment = "name"
+     *     },
+     *     {
+     *          name = "age"
+     *          type = "int"
+     *          columnLength = 0
+     *          nullable = true
+     *          defaultValue = null
+     *          comment = "age"
+     *     }
+     *     ]
+     * 
+ * + * @param columnConfig columns config + * @return columns + */ + private Column parseFromColumn(ReadonlyConfig columnConfig) { + String name = + columnConfig + .getOptional(TableSchemaOptions.ColumnOptions.NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.columns.* config need option [name], please correct your config first")); + SeaTunnelDataType seaTunnelDataType = + columnConfig + .getOptional(TableSchemaOptions.ColumnOptions.TYPE) + .map(SeaTunnelDataTypeConvertorUtil::deserializeSeaTunnelDataType) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.columns.* config need option [type], please correct your config first")); + + Integer columnLength = columnConfig.get(TableSchemaOptions.ColumnOptions.COLUMN_LENGTH); + Boolean nullable = columnConfig.get(TableSchemaOptions.ColumnOptions.NULLABLE); + Object defaultValue = columnConfig.get(TableSchemaOptions.ColumnOptions.DEFAULT_VALUE); + String comment = columnConfig.get(TableSchemaOptions.ColumnOptions.COMMENT); + return PhysicalColumn.of( + name, seaTunnelDataType, columnLength, nullable, defaultValue, comment); + } + + /** + * Parse primary key from primary key config. + * + *
+     *     primaryKey {
+     *          name = "primary_key"
+     *          columnNames = ["name", "age"]
+     *     }
+     * 
+ * + * @param primaryKeyConfig primary key config + * @return primary key + */ + private PrimaryKey parsePrimaryKey(Map primaryKeyConfig) { + if (!primaryKeyConfig.containsKey( + TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_NAME.key()) + || !primaryKeyConfig.containsKey( + TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_COLUMNS.key())) { + throw new IllegalArgumentException( + "Schema config need option [primaryKey.name, primaryKey.columnNames], please correct your config first"); + } + + String primaryKeyName = + (String) + primaryKeyConfig.get( + TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_NAME.key()); + List columns = + (List) + primaryKeyConfig.get( + TableSchemaOptions.PrimaryKeyOptions.PRIMARY_KEY_COLUMNS.key()); + return new PrimaryKey(primaryKeyName, columns); + } + + private ConstraintKey parseConstraintKeys(ReadonlyConfig constraintKeyConfig) { + String constraintName = + constraintKeyConfig + .getOptional(TableSchemaOptions.ConstraintKeyOptions.CONSTRAINT_KEY_NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [constraintName], please correct your config first")); + ConstraintKey.ConstraintType constraintType = + constraintKeyConfig + .getOptional(TableSchemaOptions.ConstraintKeyOptions.CONSTRAINT_KEY_TYPE) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [constraintType], please correct your config first")); + List columns = + constraintKeyConfig + .getOptional(TableSchemaOptions.ConstraintKeyOptions.CONSTRAINT_KEY_COLUMNS) + .map( + constraintColumnMapList -> { + return constraintColumnMapList.stream() + .map(ReadonlyConfig::fromMap) + .map( + constraintColumnConfig -> { + String columnName = + constraintColumnConfig + .getOptional( + TableSchemaOptions + .ConstraintKeyOptions + .CONSTRAINT_KEY_COLUMN_NAME) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.constraintColumns.* config need option [columnName], please correct your config first")); + ConstraintKey.ColumnSortType + columnSortType = + constraintColumnConfig.get( + TableSchemaOptions + .ConstraintKeyOptions + .CONSTRAINT_KEY_COLUMN_SORT_TYPE); + return ConstraintKey.ConstraintKeyColumn.of( + columnName, columnSortType); + }) + .collect(Collectors.toList()); + }) + .orElseThrow( + () -> + new IllegalArgumentException( + "schema.constraintKeys.* config need option [columns], please correct your config first")); + + return ConstraintKey.of(constraintType, constraintName, columns); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaOptions.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaOptions.java new file mode 100644 index 000000000000..19f858b98f8d --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaOptions.java @@ -0,0 +1,155 @@ +/* + * 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.seatunnel.api.table.catalog.schema; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; + +import java.util.List; +import java.util.Map; + +public class TableSchemaOptions { + + public static final Option> SCHEMA = + Options.key("schema") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription("SeaTunnel Schema"); + + // We should use ColumnOptions instead of FieldOptions + @Deprecated + public static class FieldOptions { + public static final Option> FIELDS = + Options.key("schema.fields") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription("SeaTunnel Schema Fields"); + } + + public static class ColumnOptions { + + // todo: how to define List> + public static final Option>> COLUMNS = + Options.key("columns") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription("SeaTunnel Schema Columns"); + + public static final Option NAME = + Options.key("name") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Column Name"); + + public static final Option TYPE = + Options.key("type") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Column Type"); + + public static final Option COLUMN_LENGTH = + Options.key("columnLength") + .intType() + .defaultValue(0) + .withDescription("SeaTunnel Schema Column Length"); + + public static final Option NULLABLE = + Options.key("nullable") + .booleanType() + .defaultValue(true) + .withDescription("SeaTunnel Schema Column Nullable"); + + public static final Option DEFAULT_VALUE = + Options.key("defaultValue") + .objectType(Object.class) + .noDefaultValue() + .withDescription("SeaTunnel Schema Column Default Value"); + + public static final Option COMMENT = + Options.key("comment") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Column Comment"); + } + + public static class PrimaryKeyOptions { + + public static final Option> PRIMARY_KEY = + Options.key("primaryKey") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription("SeaTunnel Schema Fields"); + + public static final Option PRIMARY_KEY_NAME = + Options.key("name") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Primary Key Name"); + + public static final Option> PRIMARY_KEY_COLUMNS = + Options.key("columnNames") + .listType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Primary Key Columns"); + } + + public static class ConstraintKeyOptions { + public static final Option>> CONSTRAINT_KEYS = + Options.key("constraintKeys") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription( + "SeaTunnel Schema Constraint Keys. e.g. [{name: \"xx_index\", type: \"KEY\", columnKeys: [{columnName: \"name\", sortType: \"ASC\"}]}]"); + + public static final Option CONSTRAINT_KEY_NAME = + Options.key("constraintName") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Constraint Key Name"); + + public static final Option CONSTRAINT_KEY_TYPE = + Options.key("constraintType") + .enumType(ConstraintKey.ConstraintType.class) + .noDefaultValue() + .withDescription( + "SeaTunnel Schema Constraint Key Type, e.g. KEY, UNIQUE_KEY, FOREIGN_KEY"); + + public static final Option>> CONSTRAINT_KEY_COLUMNS = + Options.key("constraintColumns") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription( + "SeaTunnel Schema Constraint Key Columns. e.g. [{columnName: \"name\", sortType: \"ASC\"}]"); + + public static final Option CONSTRAINT_KEY_COLUMN_NAME = + Options.key("columnName") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Constraint Key Column Name"); + + public static final Option CONSTRAINT_KEY_COLUMN_SORT_TYPE = + Options.key("sortType") + .enumType(ConstraintKey.ColumnSortType.class) + .defaultValue(ConstraintKey.ColumnSortType.ASC) + .withDescription( + "SeaTunnel Schema Constraint Key Column Sort Type, e.g. ASC, DESC"); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaParser.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaParser.java new file mode 100644 index 000000000000..dfd5d6d83e66 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/TableSchemaParser.java @@ -0,0 +1,81 @@ +/* + * 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.seatunnel.api.table.catalog.schema; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableSchema; + +import java.util.List; + +public interface TableSchemaParser { + + /** + * Parse schema config to TableSchema + * + * @param schemaConfig schema config + * @return TableSchema + */ + TableSchema parse(T schemaConfig); + + @Deprecated + interface FieldParser { + + /** + * Parse field config to List + * + * @param schemaConfig schema config + * @return List column list + */ + List parse(T schemaConfig); + } + + interface ColumnParser { + + /** + * Parse column config to List + * + * @param schemaConfig schema config + * @return List column list + */ + List parse(T schemaConfig); + } + + interface ConstraintKeyParser { + + /** + * Parse constraint key config to ConstraintKey + * + * @param schemaConfig schema config + * @return List constraint key list + */ + List parse(T schemaConfig); + } + + interface PrimaryKeyParser { + + /** + * Parse primary key config to PrimaryKey + * + * @param schemaConfig schema config + * @return PrimaryKey + */ + PrimaryKey parse(T schemaConfig); + } +} diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/BaseConfigParserTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/BaseConfigParserTest.java new file mode 100644 index 000000000000..9e4c33b373a6 --- /dev/null +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/BaseConfigParserTest.java @@ -0,0 +1,51 @@ +/* + * 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.seatunnel.api.table.catalog.schema; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtilTest; + +import java.io.File; +import java.io.FileNotFoundException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Paths; + +public class BaseConfigParserTest { + + protected Config getConfig(String configFile) throws FileNotFoundException, URISyntaxException { + return ConfigFactory.parseFile(new File(getTestConfigFile(configFile))); + } + + protected ReadonlyConfig getReadonlyConfig(String configFile) + throws FileNotFoundException, URISyntaxException { + return ReadonlyConfig.fromConfig(getConfig(configFile)); + } + + private String getTestConfigFile(String configFile) + throws FileNotFoundException, URISyntaxException { + URL resource = CatalogTableUtilTest.class.getResource(configFile); + if (resource == null) { + throw new FileNotFoundException("Can't find config file: " + configFile); + } + return Paths.get(resource.toURI()).toString(); + } +} diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParserTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParserTest.java new file mode 100644 index 000000000000..839dc5b1ae10 --- /dev/null +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParserTest.java @@ -0,0 +1,112 @@ +/* + * 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.seatunnel.api.table.catalog.schema; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.FileNotFoundException; +import java.net.URISyntaxException; +import java.util.List; + +class ReadonlyConfigParserTest extends BaseConfigParserTest { + + private final String columnConfig = "/conf/catalog/schema_column.conf"; + private final String fieldConfig = "/conf/catalog/schema_field.conf"; + + @Test + void parseColumn() throws FileNotFoundException, URISyntaxException { + ReadonlyConfig config = getReadonlyConfig(columnConfig); + + ReadonlyConfigParser readonlyConfigParser = new ReadonlyConfigParser(); + TableSchema tableSchema = readonlyConfigParser.parse(config); + assertPrimaryKey(tableSchema); + assertConstraintKey(tableSchema); + assertColumn(tableSchema); + } + + @Test + void parseField() throws FileNotFoundException, URISyntaxException { + ReadonlyConfig config = getReadonlyConfig(fieldConfig); + + ReadonlyConfigParser readonlyConfigParser = new ReadonlyConfigParser(); + TableSchema tableSchema = readonlyConfigParser.parse(config); + assertPrimaryKey(tableSchema); + assertConstraintKey(tableSchema); + assertColumn(tableSchema); + } + + private void assertPrimaryKey(TableSchema tableSchema) { + PrimaryKey primaryKey = tableSchema.getPrimaryKey(); + Assertions.assertEquals("id", primaryKey.getPrimaryKey()); + Assertions.assertEquals("id", primaryKey.getColumnNames().get(0)); + } + + private void assertConstraintKey(TableSchema tableSchema) { + List constraintKeys = tableSchema.getConstraintKeys(); + ConstraintKey constraintKey = constraintKeys.get(0); + Assertions.assertEquals("id_index", constraintKey.getConstraintName()); + Assertions.assertEquals( + ConstraintKey.ConstraintType.KEY, constraintKey.getConstraintType()); + Assertions.assertEquals("id", constraintKey.getColumnNames().get(0).getColumnName()); + Assertions.assertEquals( + ConstraintKey.ColumnSortType.ASC, + constraintKey.getColumnNames().get(0).getSortType()); + } + + private void assertColumn(TableSchema tableSchema) { + List columns = tableSchema.getColumns(); + Assertions.assertEquals(19, columns.size()); + + Assertions.assertEquals("id", columns.get(0).getName()); + + Assertions.assertEquals("map", columns.get(1).getName()); + Assertions.assertEquals( + "map>", + columns.get(1).getDataType().toString().toLowerCase()); + + Assertions.assertEquals("map_array", columns.get(2).getName()); + Assertions.assertEquals( + "map>>", + columns.get(2).getDataType().toString().toLowerCase()); + + Assertions.assertEquals("array", columns.get(3).getName()); + Assertions.assertEquals( + "array", columns.get(3).getDataType().toString().toLowerCase()); + + Assertions.assertEquals("string", columns.get(4).getName()); + Assertions.assertEquals("string", columns.get(4).getDataType().toString().toLowerCase()); + + Assertions.assertEquals("row", columns.get(18).getName()); + Assertions.assertEquals(SqlType.ROW, columns.get(18).getDataType().getSqlType()); + + SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) columns.get(18).getDataType(); + Assertions.assertEquals(18, seaTunnelRowType.getTotalFields()); + + SeaTunnelRowType seatunnalRowType1 = (SeaTunnelRowType) seaTunnelRowType.getFieldType(17); + Assertions.assertEquals(17, seatunnalRowType1.getTotalFields()); + } +} diff --git a/seatunnel-api/src/test/resources/conf/catalog/schema_column.conf b/seatunnel-api/src/test/resources/conf/catalog/schema_column.conf new file mode 100644 index 000000000000..486626f2df31 --- /dev/null +++ b/seatunnel-api/src/test/resources/conf/catalog/schema_column.conf @@ -0,0 +1,189 @@ +# +# 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. +# + +schema = { + columns = [ + { + name = id + type = bigint + nullable = false + defaultValue = 0 + comment = "primary key id" + }, + { + name = map + type = "map>" + nullable = true + comment = "map value" + }, + { + name = map_array + type = "map>>" + nullable = true + comment = "map_array value" + }, + { + name = array + type = "array" + nullable = true + comment = "array value" + }, + { + name = string + type = "string" + nullable = true + comment = "string value" + }, + { + name = boolean + type = "boolean" + nullable = true + comment = "boolean value" + }, + { + name = tinyint + type = "tinyint" + nullable = true + comment = "tinyint value" + }, + { + name = smallint + type = "smallint" + nullable = true + comment = "smallint value" + }, + { + name = int + type = "int" + nullable = true + comment = "int value" + }, + { + name = bigint + type = "bigint" + nullable = true + comment = "bigint value" + }, + { + name = float + type = "float" + nullable = true + comment = "float value" + }, + { + name = double + type = "double" + nullable = true + comment = "double value" + }, + { + name = decimal + type = "decimal(30, 8)" + nullable = true + comment = "decimal value" + }, + { + name = "null" + type = "null" + nullable = true + comment = "null value" + }, + { + name = bytes + type = "bytes" + nullable = true + comment = "bytes value" + }, + { + name = date + type = "date" + nullable = true + comment = "date value" + }, + { + name = time + type = "time" + nullable = true + comment = "time value" + }, + { + name = timestamp + type = "timestamp" + nullable = true + comment = "timestamp value" + }, + { + name = row + type = { + map = "map>" + map_array = "map>>" + array = "array" + string = string + boolean = boolean + tinyint = tinyint + smallint = smallint + int = int + bigint = bigint + float = float + double = double + decimal = "decimal(30, 8)" + null = "null" + bytes = bytes + date = date + time = time + timestamp = timestamp + row = { + map = "map>" + map_array = "map>>" + array = "array" + string = string + boolean = boolean + tinyint = tinyint + smallint = smallint + int = int + bigint = bigint + float = float + double = double + decimal = "decimal(30, 8)" + null = "null" + bytes = bytes + date = date + time = time + timestamp = timestamp + } + } + nullable = true + comment = "row value" + } + ] + primaryKey { + name = "id" + columnNames = [id] + } + constraintKeys = [ + { + constraintName = "id_index" + constraintType = KEY + constraintColumns = [ + { + columnName = "id" + sortType = ASC + } + ] + }, + ] +} \ No newline at end of file diff --git a/seatunnel-api/src/test/resources/conf/catalog/schema_field.conf b/seatunnel-api/src/test/resources/conf/catalog/schema_field.conf new file mode 100644 index 000000000000..75fa40af9137 --- /dev/null +++ b/seatunnel-api/src/test/resources/conf/catalog/schema_field.conf @@ -0,0 +1,93 @@ +# +# 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. +# + +schema { + fields { + id = int + map = "map>" + map_array = "map>>" + array = "array" + string = string + boolean = boolean + tinyint = tinyint + smallint = smallint + int = int + bigint = bigint + float = float + double = double + decimal = "decimal(30, 8)" + null = "null" + bytes = bytes + date = date + time = time + timestamp = timestamp + row = { + map = "map>" + map_array = "map>>" + array = "array" + string = string + boolean = boolean + tinyint = tinyint + smallint = smallint + int = int + bigint = bigint + float = float + double = double + decimal = "decimal(30, 8)" + null = "null" + bytes = bytes + date = date + time = time + timestamp = timestamp + row = { + map = "map>" + map_array = "map>>" + array = "array" + string = string + boolean = boolean + tinyint = tinyint + smallint = smallint + int = int + bigint = bigint + float = float + double = double + decimal = "decimal(30, 8)" + null = "null" + bytes = bytes + date = date + time = time + timestamp = timestamp + } + } + } + primaryKey { + name = "id" + columnNames = [id] + } + constraintKeys = [ + { + constraintName = "id_index" + constraintType = KEY + constraintColumns = [ + { + columnName = "id" + sortType = ASC + } + ] + } + ] +} \ No newline at end of file diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/TypesafeConfigUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/TypesafeConfigUtils.java index b4837552d983..c931535e186e 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/TypesafeConfigUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/TypesafeConfigUtils.java @@ -88,7 +88,10 @@ public static boolean hasSubConfig(Config source, String prefix) { @SuppressWarnings("unchecked") public static T getConfig( - final Config config, final String configKey, @NonNull final T defaultValue) { + final Config config, final String configKey, final T defaultValue) { + if (!config.hasPath(configKey) && defaultValue == null) { + return defaultValue; + } if (defaultValue.getClass().equals(Long.class)) { return config.hasPath(configKey) ? (T) Long.valueOf(config.getString(configKey)) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/JsonUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/JsonUtils.java index 9a3dbe39d1f0..84ac374b25a9 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/JsonUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/JsonUtils.java @@ -30,6 +30,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.SerializationFeature; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.SerializerProvider; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.TextNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.type.CollectionType; @@ -39,6 +40,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.TimeZone; @@ -178,6 +180,22 @@ public static Map toMap(JsonNode jsonNode) { jsonNode, new TypeReference>() {}); } + public static Map toStringMap(JsonNode jsonNode) { + Map fieldsMap = new LinkedHashMap<>(); + jsonNode.fields() + .forEachRemaining( + field -> { + String key = field.getKey(); + JsonNode value = field.getValue(); + if (value.getNodeType() == JsonNodeType.OBJECT) { + fieldsMap.put(key, value.toString()); + } else { + fieldsMap.put(key, value.textValue()); + } + }); + return fieldsMap; + } + /** * json to map * diff --git a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/config/AmazonDynamoDBSourceOptions.java b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/config/AmazonDynamoDBSourceOptions.java index 54f955f540e0..d5183c952ed4 100644 --- a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/config/AmazonDynamoDBSourceOptions.java +++ b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/config/AmazonDynamoDBSourceOptions.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import lombok.AllArgsConstructor; import lombok.Data; @@ -50,8 +50,8 @@ public AmazonDynamoDBSourceOptions(Config config) { this.accessKeyId = config.getString(AmazonDynamoDBConfig.ACCESS_KEY_ID.key()); this.secretAccessKey = config.getString(AmazonDynamoDBConfig.SECRET_ACCESS_KEY.key()); this.table = config.getString(AmazonDynamoDBConfig.TABLE.key()); - if (config.hasPath(CatalogTableUtil.SCHEMA.key())) { - this.schema = config.getConfig(CatalogTableUtil.SCHEMA.key()); + if (config.hasPath(TableSchemaOptions.SCHEMA.key())) { + this.schema = config.getConfig(TableSchemaOptions.SCHEMA.key()); } if (config.hasPath(AmazonDynamoDBConfig.BATCH_SIZE.key())) { this.batchSize = config.getInt(AmazonDynamoDBConfig.BATCH_SIZE.key()); diff --git a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSource.java b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSource.java index ccc77a1dc1d4..da84c4099bac 100644 --- a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSource.java +++ b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSource.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -40,7 +41,6 @@ import com.google.auto.service.AutoService; import lombok.extern.slf4j.Slf4j; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.amazondynamodb.config.AmazonDynamoDBConfig.ACCESS_KEY_ID; import static org.apache.seatunnel.connectors.seatunnel.amazondynamodb.config.AmazonDynamoDBConfig.REGION; import static org.apache.seatunnel.connectors.seatunnel.amazondynamodb.config.AmazonDynamoDBConfig.SECRET_ACCESS_KEY; @@ -71,7 +71,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { REGION.key(), ACCESS_KEY_ID.key(), SECRET_ACCESS_KEY.key(), - SCHEMA.key()); + TableSchemaOptions.SCHEMA.key()); if (!result.isSuccess()) { throw new AmazonDynamoDBConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSourceFactory.java b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSourceFactory.java index 4a477b625fbd..994f862908be 100644 --- a/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSourceFactory.java +++ b/seatunnel-connectors-v2/connector-amazondynamodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazondynamodb/source/AmazonDynamoDBSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -47,7 +47,7 @@ public OptionRule optionRule() { ACCESS_KEY_ID, SECRET_ACCESS_KEY, TABLE, - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/exception/AssertConnectorErrorCode.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/exception/AssertConnectorErrorCode.java index 16ae8aed1c7d..170fe2d3fd0c 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/exception/AssertConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/exception/AssertConnectorErrorCode.java @@ -21,7 +21,9 @@ public enum AssertConnectorErrorCode implements SeaTunnelErrorCode { RULE_VALIDATION_FAILED("ASSERT-01", "Rule validate failed"), - TYPES_NOT_SUPPORTED_FAILED("ASSERT-02", "Types not supported"); + TYPES_NOT_SUPPORTED_FAILED("ASSERT-02", "Types not supported"), + CATALOG_TABLE_FAILED("ASSERT-03", "Catalog table failed"), + ; private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRule.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRule.java new file mode 100644 index 000000000000..5245663b541d --- /dev/null +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRule.java @@ -0,0 +1,141 @@ +/* + * 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.seatunnel.connectors.seatunnel.assertion.rule; + +import org.apache.seatunnel.api.configuration.util.OptionMark; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.connectors.seatunnel.assertion.exception.AssertConnectorException; + +import org.apache.commons.collections4.CollectionUtils; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +import java.io.Serializable; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.assertion.exception.AssertConnectorErrorCode.CATALOG_TABLE_FAILED; + +@Data +public class AssertCatalogTableRule implements Serializable { + + @OptionMark(description = "assert primary key rule") + private AssertPrimaryKeyRule primaryKeyRule; + + @OptionMark(description = "constraint key rule") + private AssertConstraintKeyRule constraintKeyRule; + + @OptionMark(description = "column rule") + private AssertColumnRule columnRule; + + public void checkRule(CatalogTable catalogTable) { + TableSchema tableSchema = catalogTable.getTableSchema(); + if (tableSchema == null) { + throw new AssertConnectorException(CATALOG_TABLE_FAILED, "tableSchema is null"); + } + if (primaryKeyRule != null) { + primaryKeyRule.checkRule(tableSchema.getPrimaryKey()); + } + if (constraintKeyRule != null) { + constraintKeyRule.checkRule(tableSchema.getConstraintKeys()); + } + if (columnRule != null) { + columnRule.checkRule(tableSchema.getColumns()); + } + } + + @Data + @NoArgsConstructor + @AllArgsConstructor + public static class AssertPrimaryKeyRule implements Serializable { + private static final long serialVersionUID = 1L; + + @OptionMark(description = "primary key name") + private String primaryKeyName; + + @OptionMark(description = "primary key columns") + private List primaryKeyColumns; + + public void checkRule(PrimaryKey check) { + if (check == null) { + throw new AssertConnectorException(CATALOG_TABLE_FAILED, "primaryKey is null"); + } + if (primaryKeyName != null && !primaryKeyName.equals(check.getPrimaryKey())) { + throw new AssertConnectorException( + CATALOG_TABLE_FAILED, + String.format( + "primaryKey: %s is not equal to %s", + check.getPrimaryKey(), primaryKeyName)); + } + if (CollectionUtils.isNotEmpty(primaryKeyColumns) + && !CollectionUtils.isEqualCollection( + primaryKeyColumns, check.getColumnNames())) { + throw new AssertConnectorException( + CATALOG_TABLE_FAILED, + String.format( + "primaryKey columns: %s is not equal to %s", + check.getColumnNames(), primaryKeyColumns)); + } + } + } + + @Data + @AllArgsConstructor + public static class AssertConstraintKeyRule implements Serializable { + private static final long serialVersionUID = 1L; + private List constraintKeys; + + public void checkRule(List check) { + if (CollectionUtils.isEmpty(check)) { + throw new AssertConnectorException(CATALOG_TABLE_FAILED, "constraintKeys is null"); + } + if (CollectionUtils.isNotEmpty(constraintKeys) + && !CollectionUtils.isEqualCollection(constraintKeys, check)) { + throw new AssertConnectorException( + CATALOG_TABLE_FAILED, + String.format( + "constraintKeys: %s is not equal to %s", check, constraintKeys)); + } + } + } + + @Data + @AllArgsConstructor + public static class AssertColumnRule implements Serializable { + private static final long serialVersionUID = 1L; + + private List columns; + + public void checkRule(List check) { + if (CollectionUtils.isEmpty(check)) { + throw new AssertConnectorException(CATALOG_TABLE_FAILED, "columns is null"); + } + if (CollectionUtils.isNotEmpty(columns) + && !CollectionUtils.isEqualCollection(columns, check)) { + throw new AssertConnectorException( + CATALOG_TABLE_FAILED, + String.format("columns: %s is not equal to %s", check, columns)); + } + } + } +} diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRuleParser.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRuleParser.java new file mode 100644 index 000000000000..dc5063a2c891 --- /dev/null +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertCatalogTableRuleParser.java @@ -0,0 +1,159 @@ +/* + * 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.seatunnel.connectors.seatunnel.assertion.rule; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.SeaTunnelDataTypeConvertorUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.common.config.TypesafeConfigUtils; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_COMMENT; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_DEFAULT_VALUE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_LENGTH; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_NAME; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_NULLABLE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_RULE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.COLUMN_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_COLUMNS; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_COLUMN_NAME; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_NAME; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_RULE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_SORT_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CONSTRAINT_KEY_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.PRIMARY_KEY_COLUMNS; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.PRIMARY_KEY_NAME; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.PRIMARY_KEY_RULE; + +public class AssertCatalogTableRuleParser { + + public AssertCatalogTableRule parseCatalogTableRule(Config catalogTableRule) { + AssertCatalogTableRule tableRule = new AssertCatalogTableRule(); + + parsePrimaryKeyRule(catalogTableRule).ifPresent(tableRule::setPrimaryKeyRule); + parseConstraintKeyRule(catalogTableRule).ifPresent(tableRule::setConstraintKeyRule); + parseColumnRule(catalogTableRule).ifPresent(tableRule::setColumnRule); + return tableRule; + } + + private Optional parsePrimaryKeyRule( + Config catalogTableRule) { + if (!catalogTableRule.hasPath(PRIMARY_KEY_RULE)) { + return Optional.empty(); + } + Config primaryKey = catalogTableRule.getConfig(PRIMARY_KEY_RULE); + return Optional.of( + new AssertCatalogTableRule.AssertPrimaryKeyRule( + primaryKey.getString(PRIMARY_KEY_NAME), + primaryKey.getStringList(PRIMARY_KEY_COLUMNS))); + } + + private Optional parseColumnRule( + Config catalogTableRule) { + if (!catalogTableRule.hasPath(COLUMN_RULE)) { + return Optional.empty(); + } + List columns = + catalogTableRule.getConfigList(COLUMN_RULE).stream() + .map( + config -> { + String name = config.getString(COLUMN_NAME); + String type = config.getString(COLUMN_TYPE); + Integer columnLength = + TypesafeConfigUtils.getConfig( + config, + COLUMN_LENGTH, + TableSchemaOptions.ColumnOptions.COLUMN_LENGTH + .defaultValue()); + Boolean nullable = + TypesafeConfigUtils.getConfig( + config, + COLUMN_NULLABLE, + TableSchemaOptions.ColumnOptions.NULLABLE + .defaultValue()); + Object object = + TypesafeConfigUtils.getConfig( + config, + COLUMN_DEFAULT_VALUE, + TableSchemaOptions.ColumnOptions.DEFAULT_VALUE + .defaultValue()); + String comment = + TypesafeConfigUtils.getConfig( + config, + COLUMN_COMMENT, + TableSchemaOptions.ColumnOptions.COMMENT + .defaultValue()); + return PhysicalColumn.of( + name, + SeaTunnelDataTypeConvertorUtil + .deserializeSeaTunnelDataType(type), + columnLength, + nullable, + object, + comment); + }) + .collect(Collectors.toList()); + return Optional.of(new AssertCatalogTableRule.AssertColumnRule(columns)); + } + + private Optional parseConstraintKeyRule( + Config catalogTableRule) { + if (!catalogTableRule.hasPath(CONSTRAINT_KEY_RULE)) { + return Optional.empty(); + } + List constraintKey = catalogTableRule.getConfigList(CONSTRAINT_KEY_RULE); + List constraintKeys = + constraintKey.stream() + .map( + config -> { + ConstraintKey.ConstraintType constraintType = + ConstraintKey.ConstraintType.valueOf( + config.getString(CONSTRAINT_KEY_TYPE)); + String constraintKeyName = + config.getString(CONSTRAINT_KEY_NAME); + List constraintKeyColumns = + config.getConfigList(CONSTRAINT_KEY_COLUMNS).stream() + .map( + c -> + ConstraintKey + .ConstraintKeyColumn.of( + c.getString( + CONSTRAINT_KEY_COLUMN_NAME), + ConstraintKey + .ColumnSortType + .valueOf( + c + .getString( + CONSTRAINT_KEY_SORT_TYPE)))) + .collect(Collectors.toList()); + return ConstraintKey.of( + constraintType, + constraintKeyName, + constraintKeyColumns); + }) + .collect(Collectors.toList()); + return Optional.of(new AssertCatalogTableRule.AssertConstraintKeyRule(constraintKeys)); + } +} diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertRuleParser.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertRuleParser.java index eccf2c684505..5d90591af7ca 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertRuleParser.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/rule/AssertRuleParser.java @@ -44,6 +44,10 @@ public List parseRowRules(List row return assembleFieldValueRules(rowRuleList); } + public AssertCatalogTableRule parseCatalogTableRule(Config catalogTableRule) { + return new AssertCatalogTableRuleParser().parseCatalogTableRule(catalogTableRule); + } + public List parseRules(List ruleConfigList) { return ruleConfigList.stream() .map( diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertConfig.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertConfig.java index 3e3da9f68b9b..5d9dddc58d78 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertConfig.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertConfig.java @@ -17,9 +17,13 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import java.util.Map; + public class AssertConfig { public static final String RULE_TYPE = "rule_type"; @@ -38,9 +42,37 @@ public class AssertConfig { public static final String FIELD_RULES = "field_rules"; - public static final Option RULES = + public static final String CATALOG_TABLE_RULES = "catalog_table_rule"; + + public static final String PRIMARY_KEY_RULE = "primary_key_rule"; + public static final String PRIMARY_KEY_NAME = "primary_key_name"; + public static final String PRIMARY_KEY_COLUMNS = "primary_key_columns"; + + public static final String CONSTRAINT_KEY_RULE = "constraint_key_rule"; + public static final String CONSTRAINT_KEY_NAME = "constraint_key_name"; + public static final String CONSTRAINT_KEY_TYPE = "constraint_key_type"; + public static final String CONSTRAINT_KEY_COLUMNS = "constraint_key_columns"; + public static final String CONSTRAINT_KEY_COLUMN_NAME = "constraint_key_column_name"; + public static final String CONSTRAINT_KEY_SORT_TYPE = "constraint_key_sort_type"; + + public static final String COLUMN_RULE = "column_rule"; + + public static final String COLUMN_NAME = "name"; + public static final String COLUMN_TYPE = "type"; + public static final String COLUMN_LENGTH = "column_length"; + public static final String COLUMN_NULLABLE = "nullable"; + public static final String COLUMN_DEFAULT_VALUE = "default_value"; + public static final String COLUMN_COMMENT = "comment"; + + public static final Option COMMENT = + Options.key("comment") + .stringType() + .noDefaultValue() + .withDescription("SeaTunnel Schema Column Comment"); + + public static final Option> RULES = Options.key("rules") - .objectType(Rules.class) + .type(new TypeReference>() {}) .noDefaultValue() .withDescription( "Rule definition of user's available data. Each rule represents one field validation or row num validation."); diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java index cb15f16101f4..b4ff1e9a8530 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java @@ -17,51 +17,91 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigException; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.CATALOG_TABLE_RULES; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.FIELD_RULES; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.ROW_RULES; +import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.RULES; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.assertion.rule.AssertCatalogTableRule; import org.apache.seatunnel.connectors.seatunnel.assertion.rule.AssertFieldRule; import org.apache.seatunnel.connectors.seatunnel.assertion.rule.AssertRuleParser; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigException; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; -import org.apache.commons.collections4.CollectionUtils; +import java.util.List; import com.google.auto.service.AutoService; import com.google.common.base.Throwables; -import java.io.IOException; -import java.util.List; - -import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.FIELD_RULES; -import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.ROW_RULES; -import static org.apache.seatunnel.connectors.seatunnel.assertion.sink.AssertConfig.RULES; - @AutoService(SeaTunnelSink.class) public class AssertSink extends AbstractSimpleSink { - private SeaTunnelRowType seaTunnelRowType; + private CatalogTable catalogTable; private List assertFieldRules; private List assertRowRules; + private AssertCatalogTableRule assertCatalogTableRule; + + public AssertSink() { + } + + public AssertSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { + this.catalogTable = catalogTable; + if (!pluginConfig.getOptional(RULES).isPresent()) { + Throwables.propagateIfPossible(new ConfigException.Missing(RULES.key())); + } + Config ruleConfig = ConfigFactory.parseMap(pluginConfig.get(RULES, false)); + List rowConfigList = null; + List configList = null; + if (ruleConfig.hasPath(ROW_RULES)) { + rowConfigList = ruleConfig.getConfigList(ROW_RULES); + assertRowRules = new AssertRuleParser().parseRowRules(rowConfigList); + } + if (ruleConfig.hasPath(FIELD_RULES)) { + configList = ruleConfig.getConfigList(FIELD_RULES); + assertFieldRules = new AssertRuleParser().parseRules(configList); + } + + if (ruleConfig.hasPath(CATALOG_TABLE_RULES)) { + assertCatalogTableRule = + new AssertRuleParser() + .parseCatalogTableRule(ruleConfig.getConfig(CATALOG_TABLE_RULES)); + assertCatalogTableRule.checkRule(catalogTable); + } + + if (CollectionUtils.isEmpty(configList) + && CollectionUtils.isEmpty(rowConfigList) + && assertCatalogTableRule == null) { + Throwables.propagateIfPossible( + new ConfigException.BadValue( + RULES.key(), "Assert rule config is empty, please add rule config.")); + } + } + @Override public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { - this.seaTunnelRowType = seaTunnelRowType; + throw new UnsupportedOperationException("Please use CatalogTable to set type info."); } @Override public SeaTunnelDataType getConsumedType() { - return this.seaTunnelRowType; + return this.catalogTable.getSeaTunnelRowType(); } @Override - public AbstractSinkWriter createWriter(SinkWriter.Context context) - throws IOException { - return new AssertSinkWriter(seaTunnelRowType, assertFieldRules, assertRowRules); + public AbstractSinkWriter createWriter(SinkWriter.Context context) { + return new AssertSinkWriter( + catalogTable.getSeaTunnelRowType(), assertFieldRules, assertRowRules); } @Override @@ -83,8 +123,8 @@ public void prepare(Config pluginConfig) { if (CollectionUtils.isEmpty(configList) && CollectionUtils.isEmpty(rowConfigList)) { Throwables.propagateIfPossible( - new ConfigException.BadValue( - RULES.key(), "Assert rule config is empty, please add rule config.")); + new ConfigException.BadValue( + RULES.key(), "Assert rule config is empty, please add rule config.")); } } diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java index 66588a92414c..376863dc184e 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java @@ -18,8 +18,10 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; import com.google.auto.service.AutoService; @@ -37,4 +39,9 @@ public String factoryIdentifier() { public OptionRule optionRule() { return OptionRule.builder().required(RULES).build(); } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + return () -> new AssertSink(context.getOptions(), context.getCatalogTable()); + } } diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/Rules.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/Rules.java index 0f974868cb49..6b4fb8714738 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/Rules.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/Rules.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; import org.apache.seatunnel.api.configuration.util.OptionMark; +import org.apache.seatunnel.connectors.seatunnel.assertion.rule.AssertCatalogTableRule; import lombok.Data; @@ -31,4 +32,7 @@ public class Rules { @OptionMark(description = "field rules for field validation") private List fieldRules; + + @OptionMark(description = "catalog table rule for catalog table validation") + private AssertCatalogTableRule catalogTableRule; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java index c1878a255eff..e577695e9fb2 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -61,7 +62,7 @@ public String getPluginName() { @Override public void prepare(Config pluginConfig) throws PrepareFailException { this.pluginConfig = pluginConfig; - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { // todo: We need to remove the schema in ES. rowTypeInfo = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); source = Arrays.asList(rowTypeInfo.getFieldNames()); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java index 5c4d71288439..26ebb8049cbf 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -63,7 +63,7 @@ public OptionRule optionRule() { TLS_KEY_STORE_PASSWORD, TLS_TRUST_STORE_PATH, TLS_TRUST_STORE_PASSWORD) - .exclusive(SOURCE, CatalogTableUtil.SCHEMA) + .exclusive(SOURCE, TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java index cd8fbda618c3..bd8c3869d2a3 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java @@ -18,16 +18,20 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; @@ -38,20 +42,26 @@ import org.apache.seatunnel.connectors.seatunnel.fake.exception.FakeConnectorException; import org.apache.seatunnel.connectors.seatunnel.fake.state.FakeSourceState; -import com.google.auto.service.AutoService; +import com.google.common.collect.Lists; import java.util.Collections; +import java.util.List; -@AutoService(SeaTunnelSource.class) public class FakeSource implements SeaTunnelSource, SupportParallelism, SupportColumnProjection { private JobContext jobContext; - private SeaTunnelRowType rowType; + private CatalogTable catalogTable; private FakeConfig fakeConfig; + public FakeSource(ReadonlyConfig readonlyConfig) { + this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); + this.fakeConfig = + FakeConfig.buildWithConfig(ConfigFactory.parseMap(readonlyConfig.toMap())); + } + @Override public Boundedness getBoundedness() { return JobMode.BATCH.equals(jobContext.getJobMode()) @@ -59,9 +69,14 @@ public Boundedness getBoundedness() { : Boundedness.UNBOUNDED; } + @Override + public List getProducedCatalogTables() { + return Lists.newArrayList(catalogTable); + } + @Override public SeaTunnelRowType getProducedType() { - return rowType; + return catalogTable.getSeaTunnelRowType(); } @Override @@ -73,16 +88,15 @@ public SourceSplitEnumerator createEnumerator( @Override public SourceSplitEnumerator restoreEnumerator( SourceSplitEnumerator.Context enumeratorContext, - FakeSourceState checkpointState) - throws Exception { + FakeSourceState checkpointState) { return new FakeSourceSplitEnumerator( enumeratorContext, fakeConfig, checkpointState.getAssignedSplits()); } @Override public SourceReader createReader( - SourceReader.Context readerContext) throws Exception { - return new FakeSourceReader(readerContext, rowType, fakeConfig); + SourceReader.Context readerContext) { + return new FakeSourceReader(readerContext, catalogTable.getSeaTunnelRowType(), fakeConfig); } @Override @@ -93,7 +107,7 @@ public String getPluginName() { @Override public void prepare(Config pluginConfig) { CheckResult result = - CheckConfigUtil.checkAllExists(pluginConfig, CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(pluginConfig, TableSchemaOptions.SCHEMA.key()); if (!result.isSuccess()) { throw new FakeConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, @@ -101,8 +115,6 @@ public void prepare(Config pluginConfig) { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SOURCE, result.getMsg())); } - this.rowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); - this.fakeConfig = FakeConfig.buildWithConfig(pluginConfig); } @Override diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceFactory.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceFactory.java index e954d4e21089..ee668fa9298d 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceFactory.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceFactory.java @@ -19,13 +19,18 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeOption; import com.google.auto.service.AutoService; +import java.io.Serializable; + import static org.apache.seatunnel.connectors.seatunnel.fake.config.FakeOption.ARRAY_SIZE; import static org.apache.seatunnel.connectors.seatunnel.fake.config.FakeOption.BIGINT_FAKE_MODE; import static org.apache.seatunnel.connectors.seatunnel.fake.config.FakeOption.BIGINT_TEMPLATE; @@ -64,7 +69,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(CatalogTableUtil.SCHEMA) + .required(TableSchemaOptions.SCHEMA) .optional(STRING_FAKE_MODE) .conditional(STRING_FAKE_MODE, FakeOption.FakeMode.TEMPLATE, STRING_TEMPLATE) .optional(TINYINT_FAKE_MODE) @@ -96,6 +101,13 @@ public OptionRule optionRule() { .build(); } + @Override + @SuppressWarnings("unchecked") + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new FakeSource(context.getOptions()); + } + @Override public Class getSourceClass() { return FakeSource.class; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java index 57d2ceca6eb4..987b861d3f49 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -87,7 +88,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { .getString(HdfsSourceConfig.FILE_FORMAT_TYPE.key()) .toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java index aefc339121e0..f874666c63d6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -83,7 +84,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { FileFormat.valueOf( pluginConfig.getString(CosConfig.FILE_FORMAT_TYPE.key()).toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java index 496e9277f4e3..1e7b6a0198f1 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -55,7 +55,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java index 0c35e50f90c7..e396e2834eff 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -88,7 +89,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } // support user-defined schema // only json type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java index 4ab637c43484..e40938a329eb 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -55,7 +55,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java index c3d406d62c79..2e860df79525 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -52,7 +52,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java index e00b7abc286c..7714e62bf306 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -84,7 +85,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { FileFormat.valueOf( pluginConfig.getString(OssConfig.FILE_FORMAT_TYPE.key()).toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java index eaea7bccb61e..ab0a44d750ea 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -55,7 +55,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSource.java index 5bbf6e3e1eac..d7ba8d72249f 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -85,7 +86,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { .getString(LocalSourceConfig.FILE_FORMAT_TYPE.key()) .toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java index 03ec8660ce2d..3c2757eb4418 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -51,7 +51,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java index d4076775a0d3..3f8144c7d3fb 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -83,7 +84,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { FileFormat.valueOf( pluginConfig.getString(OssConfig.FILE_FORMAT_TYPE.key()).toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java index e7d862bd44ad..11b0b06dc7d2 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -55,7 +55,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java index 5e559e06a009..fb3eb848a138 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -80,7 +81,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { FileFormat.valueOf( pluginConfig.getString(S3Config.FILE_FORMAT_TYPE.key()).toUpperCase()); // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java index a3b48088650c..8fc55d2681c2 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -60,7 +60,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSource.java index fd9d487e2110..e7d2c86f5f08 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSource.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -88,7 +89,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } // support user-defined schema // only json csv text type support user-defined schema now - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { switch (fileFormat) { case CSV: case TEXT: diff --git a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java index e9efe1cdf9b3..8d60c01d2092 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; @@ -55,7 +55,7 @@ public OptionRule optionRule() { BaseSourceConfig.FILE_FORMAT_TYPE, Arrays.asList( FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java index 376cee6303ba..0661f6db3666 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -64,7 +65,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { SheetsConfig.SHEET_ID.key(), SheetsConfig.SHEET_NAME.key(), SheetsConfig.RANGE.key(), - CatalogTableUtil.SCHEMA.key()); + TableSchemaOptions.SCHEMA.key()); if (!checkResult.isSuccess()) { throw new GoogleSheetsConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, @@ -73,7 +74,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { getPluginName(), PluginType.SOURCE, checkResult.getMsg())); } this.sheetsParameters = new SheetsParameters().buildWithConfig(pluginConfig); - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { this.seaTunnelRowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); } else { diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSourceFactory.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSourceFactory.java index dd1f8aeb481c..192c4eaf6a70 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSourceFactory.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.google.sheets.config.SheetsConfig; @@ -40,7 +40,7 @@ public OptionRule optionRule() { .required(SheetsConfig.SHEET_ID) .required(SheetsConfig.SHEET_NAME) .required(SheetsConfig.RANGE) - .optional(CatalogTableUtil.SCHEMA) + .optional(TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java index e01efde48bc4..baabac6fe748 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SqlType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; @@ -52,7 +53,6 @@ import java.util.Map; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig.FILE_FORMAT_TYPE; import static org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig.FILE_PATH; import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.ORC_INPUT_FORMAT_CLASSNAME; @@ -83,7 +83,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { result = CheckConfigUtil.checkAtLeastOneExists( pluginConfig, - SCHEMA.key(), + TableSchemaOptions.SCHEMA.key(), FILE_FORMAT_TYPE.key(), FILE_PATH.key(), FS_DEFAULT_NAME_KEY); @@ -94,7 +94,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "Hive source connector does not support these setting [%s]", String.join( ",", - SCHEMA.key(), + TableSchemaOptions.SCHEMA.key(), FILE_FORMAT_TYPE.key(), FILE_PATH.key(), FS_DEFAULT_NAME_KEY))); @@ -135,7 +135,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { ConfigRenderOptions options = ConfigRenderOptions.concise(); String render = pluginConfig.root().render(options); ObjectNode jsonNodes = JsonUtils.parseObject(render); - jsonNodes.putPOJO(SCHEMA.key(), schema); + jsonNodes.putPOJO(TableSchemaOptions.SCHEMA.key(), schema); pluginConfig = ConfigFactory.parseString(jsonNodes.toString()); } else if (PARQUET_INPUT_FORMAT_CLASSNAME.equals(inputFormat)) { pluginConfig = diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 00e8f24b7506..8e8311b6505a 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -85,7 +86,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } protected void buildSchemaWithConfig(Config pluginConfig) { - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { this.rowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); // default use json format HttpConfig.ResponseFormat format = HttpConfig.FORMAT.defaultValue(); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceFactory.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceFactory.java index 230e84a4172f..21bc3940e1f1 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceFactory.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.http.config.HttpConfig; @@ -51,7 +51,9 @@ public OptionRule.Builder getHttpBuilder() { .optional(HttpConfig.CONTENT_FIELD) .conditional(HttpConfig.METHOD, HttpRequestMethod.POST, HttpConfig.BODY) .conditional( - HttpConfig.FORMAT, HttpConfig.ResponseFormat.JSON, CatalogTableUtil.SCHEMA) + HttpConfig.FORMAT, + HttpConfig.ResponseFormat.JSON, + TableSchemaOptions.SCHEMA) .optional(HttpConfig.POLL_INTERVAL_MILLS) .optional(HttpConfig.RETRY) .optional(HttpConfig.RETRY_BACKOFF_MULTIPLIER_MS) diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java index 246aa7dd1476..13c6575e7d1a 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -104,7 +105,7 @@ private SeaTunnelRowType loadSeaTunnelRowType(Schema tableSchema, Config pluginC columnDataTypes.toArray(new SeaTunnelDataType[0])); CheckResult checkResult = - CheckConfigUtil.checkAllExists(pluginConfig, CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(pluginConfig, TableSchemaOptions.SCHEMA.key()); if (checkResult.isSuccess()) { SeaTunnelRowType projectedRowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java index fdc6fa08858b..1318ab8a9d72 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -55,7 +55,7 @@ public OptionRule optionRule() { KEY_CATALOG_NAME, KEY_CATALOG_TYPE, KEY_WAREHOUSE, KEY_NAMESPACE, KEY_TABLE) .conditional(KEY_CATALOG_TYPE, HIVE, KEY_URI) .optional( - CatalogTableUtil.SCHEMA, + TableSchemaOptions.SCHEMA, KEY_CASE_SENSITIVE, KEY_START_SNAPSHOT_TIMESTAMP, KEY_START_SNAPSHOT_ID, diff --git a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSource.java b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSource.java index 14cae9a77a3b..c979653525c6 100644 --- a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSource.java +++ b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSource.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -75,7 +76,7 @@ public String getPluginName() { @Override public void prepare(Config config) throws PrepareFailException { CheckResult result = - CheckConfigUtil.checkAllExists(config, SQL.key(), CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(config, SQL.key(), TableSchemaOptions.SCHEMA.key()); if (!result.isSuccess()) { throw new InfluxdbConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSourceFactory.java b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSourceFactory.java index 1452266e1305..0b1539a72ecd 100644 --- a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSourceFactory.java +++ b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/source/InfluxDBSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -48,7 +48,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(URL, SQL, DATABASES, CatalogTableUtil.SCHEMA) + .required(URL, SQL, DATABASES, TableSchemaOptions.SCHEMA) .bundled(USERNAME, PASSWORD) .bundled(LOWER_BOUND, UPPER_BOUND, PARTITION_NUM, SPLIT_COLUMN) .optional(EPOCH, CONNECT_TIMEOUT_MS, QUERY_TIMEOUT_SEC) diff --git a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java index 0c171ada4fcf..6daf68f4c4cf 100644 --- a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java +++ b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java @@ -29,6 +29,7 @@ import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -66,7 +67,7 @@ public String getPluginName() { public void prepare(Config pluginConfig) throws PrepareFailException { CheckResult urlCheckResult = CheckConfigUtil.checkAllExists(pluginConfig, NODE_URLS.key()); CheckResult schemaCheckResult = - CheckConfigUtil.checkAllExists(pluginConfig, CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(pluginConfig, TableSchemaOptions.SCHEMA.key()); CheckResult mergedConfigCheck = CheckConfigUtil.mergeCheckResults(urlCheckResult, schemaCheckResult); if (!mergedConfigCheck.isSuccess()) { diff --git a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSourceFactory.java b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSourceFactory.java index 2c2a521fd84a..408d64ba3d95 100644 --- a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSourceFactory.java +++ b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSourceFactory.java @@ -19,12 +19,12 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.iotdb.config.CommonConfig.NODE_URLS; import static org.apache.seatunnel.connectors.seatunnel.iotdb.config.CommonConfig.PASSWORD; import static org.apache.seatunnel.connectors.seatunnel.iotdb.config.CommonConfig.USERNAME; @@ -48,7 +48,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(NODE_URLS, USERNAME, PASSWORD, SQL, SCHEMA) + .required(NODE_URLS, USERNAME, PASSWORD, SQL, TableSchemaOptions.SCHEMA) .optional( FETCH_SIZE, THRIFT_DEFAULT_BUFFER_SIZE, diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java index 50bb48e98a83..597ca8326629 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java @@ -18,7 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.mongodb.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; @@ -41,7 +41,7 @@ public OptionRule optionRule() { MongodbConfig.URI, MongodbConfig.DATABASE, MongodbConfig.COLLECTION, - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional( MongodbConfig.BUFFER_FLUSH_INTERVAL, MongodbConfig.BUFFER_FLUSH_MAX_ROWS, diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java index d611a9bd53b3..d9be1457f4c0 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -87,7 +88,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { .collection(collection) .build(); } - if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { this.rowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); } else { this.rowType = CatalogTableUtil.buildSimpleTextSchema(); diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java index 22687c1bf720..8d0ce28da2b7 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -46,7 +46,7 @@ public OptionRule optionRule() { MongodbConfig.URI, MongodbConfig.DATABASE, MongodbConfig.COLLECTION, - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional( MongodbConfig.PROJECTION, MongodbConfig.MATCH_QUERY, diff --git a/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSource.java b/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSource.java index 12100afe2a68..c29b46717ac6 100644 --- a/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSource.java +++ b/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSource.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -58,7 +59,7 @@ public String getPluginName() { public void prepare(Config pluginConfig) throws PrepareFailException { final CheckResult configCheck = - CheckConfigUtil.checkAllExists(pluginConfig, CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(pluginConfig, TableSchemaOptions.SCHEMA.key()); if (!configCheck.isSuccess()) { throw new Neo4jConnectorException( diff --git a/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSourceFactory.java b/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSourceFactory.java index e5ab2e8c5fc3..fd2201aec2bc 100644 --- a/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSourceFactory.java +++ b/seatunnel-connectors-v2/connector-neo4j/src/main/java/org/apache/seatunnel/connectors/seatunnel/neo4j/source/Neo4jSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -46,7 +46,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(KEY_NEO4J_URI, KEY_DATABASE, KEY_QUERY, CatalogTableUtil.SCHEMA) + .required(KEY_NEO4J_URI, KEY_DATABASE, KEY_QUERY, TableSchemaOptions.SCHEMA) .optional( KEY_USERNAME, KEY_PASSWORD, diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSourceFactory.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSourceFactory.java index fdf9fb85f0d6..9de7773cac59 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSourceFactory.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.pulsar.config.PulsarConfigUtil; @@ -62,7 +62,7 @@ public OptionRule optionRule() { POLL_TIMEOUT, POLL_INTERVAL, POLL_BATCH_SIZE, - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .exclusive(TOPIC, TOPIC_PATTERN) .conditional( CURSOR_STARTUP_MODE, diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java index d6cc201a47bd..e16d098d6c96 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/format/PulsarCanalDecorator.java @@ -58,7 +58,7 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { @Override public void deserialize(byte[] message, Collector out) throws IOException { - ObjectNode pulsarCanal = JsonUtils.parseObject(message); + JsonNode pulsarCanal = JsonUtils.parseObject(message); ArrayNode canalList = JsonUtils.parseArray(pulsarCanal.get(MESSAGE).asText()); Iterator canalIterator = canalList.elements(); while (canalIterator.hasNext()) { diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java index 24edd0e547b4..049c90c3ecda 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java @@ -29,6 +29,7 @@ import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -44,7 +45,6 @@ import com.google.auto.service.AutoService; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.HOST; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.PASSWORD; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.PORT; @@ -89,7 +89,7 @@ public void prepare(Config config) throws PrepareFailException { USERNAME.key(), PASSWORD.key(), QUEUE_NAME.key(), - SCHEMA.key()); + TableSchemaOptions.SCHEMA.key()); if (!result.isSuccess()) { throw new RabbitmqConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceFactory.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceFactory.java index d1d2d02c562e..b809a6039f27 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceFactory.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSourceFactory.java @@ -19,12 +19,12 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.AUTOMATIC_RECOVERY_ENABLED; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.CONNECTION_TIMEOUT; import static org.apache.seatunnel.connectors.seatunnel.rabbitmq.config.RabbitmqConfig.DELIVERY_TIMEOUT; @@ -54,7 +54,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(HOST, PORT, VIRTUAL_HOST, QUEUE_NAME, SCHEMA) + .required(HOST, PORT, VIRTUAL_HOST, QUEUE_NAME, TableSchemaOptions.SCHEMA) .bundled(USERNAME, PASSWORD) .optional( URL, diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index 6c0b8dd0e93b..c7ad6e6de892 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -72,7 +73,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { // TODO: use format SPI // default use json format if (pluginConfig.hasPath(RedisConfig.FORMAT.key())) { - if (!pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + if (!pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { throw new RedisConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, String.format( diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceFactory.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceFactory.java index e1f7dc902d0e..02cab4a5587f 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceFactory.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSourceFactory.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.redis.config.RedisConfig; @@ -45,7 +45,7 @@ public OptionRule optionRule() { RedisConfig.USER, RedisConfig.KEY_PATTERN) .conditional(RedisConfig.MODE, RedisConfig.RedisMode.CLUSTER, RedisConfig.NODES) - .bundled(RedisConfig.FORMAT, CatalogTableUtil.SCHEMA) + .bundled(RedisConfig.FORMAT, TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSource.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSource.java index bf468a65ee40..738400436a3f 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSource.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSource.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -67,7 +68,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { PASSWORD.key()); CheckResult schemaCheckResult = - CheckConfigUtil.checkAllExists(pluginConfig, CatalogTableUtil.SCHEMA.key()); + CheckConfigUtil.checkAllExists(pluginConfig, TableSchemaOptions.SCHEMA.key()); CheckResult mergedConfigCheck = CheckConfigUtil.mergeCheckResults(checkResult, schemaCheckResult); if (!mergedConfigCheck.isSuccess()) { diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSourceFactory.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSourceFactory.java index 52cccc045f87..fffb5a435cec 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSourceFactory.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/source/StarRocksSourceFactory.java @@ -19,14 +19,13 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.connectors.seatunnel.starrocks.config.SourceConfig; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.api.table.catalog.CatalogTableUtil.SCHEMA; - @AutoService(Factory.class) public class StarRocksSourceFactory implements TableSourceFactory { @Override @@ -43,7 +42,7 @@ public OptionRule optionRule() { SourceConfig.PASSWORD, SourceConfig.DATABASE, SourceConfig.TABLE, - SCHEMA) + TableSchemaOptions.SCHEMA) .optional( SourceConfig.MAX_RETRIES, SourceConfig.QUERY_TABLET_SIZE, diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkFactory.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkFactory.java index 674f641ad648..a3e40e622999 100644 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkFactory.java +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkFactory.java @@ -18,7 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.tablestore.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -49,7 +49,7 @@ public OptionRule optionRule() { ACCESS_KEY_ID, ACCESS_KEY_SECRET, PRIMARY_KEYS, - CatalogTableUtil.SCHEMA) + TableSchemaOptions.SCHEMA) .optional(BATCH_SIZE) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java new file mode 100644 index 000000000000..d449add802a3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.fake; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; + +import java.io.IOException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support CatalogTable") +public class FakeWithSchemaTT extends TestSuiteBase { + @TestTemplate + public void testFakeConnector(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult fakeWithCatalogTable = + container.executeJob("/fake_to_assert_with_catalogtable.conf"); + Assertions.assertEquals(0, fakeWithCatalogTable.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/resources/fake_to_assert_with_catalogtable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/resources/fake_to_assert_with_catalogtable.conf new file mode 100644 index 000000000000..4a98662b76d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/resources/fake_to_assert_with_catalogtable.conf @@ -0,0 +1,99 @@ +# +# 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. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 100 + schema = { + columns = [ + { + name = id + type = bigint + } + { + name = name + type = string + } + { + name = age + type = int + } + ] + primaryKey = { + name = "primary key" + columnNames = ["id"] + } + constraintKeys = [ + { + constraintName = "unique_name" + constraintType = UNIQUE_KEY + constraintColumns = [ + { + columnName = "id" + sortType = ASC + } + ] + } + ] + } + result_table_name = "fake" + } +} + +sink{ + Assert { + rules { + catalog_table_rule { + primary_key_rule = { + primary_key_name = "primary key" + primary_key_columns = ["id"] + } + constraint_key_rule = [ + { + constraint_key_name = "unique_name" + constraint_key_type = UNIQUE_KEY + constraint_key_columns = [ + { + constraint_key_column_name = "id" + constraint_key_sort_type = ASC + } + ] + } + ] + column_rule = [ + { + name = "id" + type = bigint + }, + { + name = "name" + type = string + }, + { + name = "age" + type = int + } + ] + } + } + } +} \ No newline at end of file diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TestUtils.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TestUtils.java index 571a3c95db47..0e47404b8e59 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TestUtils.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TestUtils.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.common.config.DeployMode; import org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSink; @@ -54,14 +55,14 @@ public static String getResource(String confFile) { public static LogicalDag getTestLogicalDag(JobContext jobContext) throws MalformedURLException { IdGenerator idGenerator = new IdGenerator(); - FakeSource fakeSource = new FakeSource(); - fakeSource.setJobContext(jobContext); Config fakeSourceConfig = ConfigFactory.parseMap( Collections.singletonMap( "schema", Collections.singletonMap( "fields", ImmutableMap.of("id", "int", "name", "string")))); + FakeSource fakeSource = new FakeSource(ReadonlyConfig.fromConfig(fakeSourceConfig)); + fakeSource.setJobContext(jobContext); fakeSource.prepare(fakeSourceConfig); Action fake = diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointPlanTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointPlanTest.java index b8b594a68b82..069c6a862685 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointPlanTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointPlanTest.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSink; import org.apache.seatunnel.connectors.seatunnel.fake.source.FakeSource; @@ -107,13 +108,13 @@ private static void fillVirtualVertex( IdGenerator idGenerator, LogicalDag logicalDag, int parallelism) { JobContext jobContext = new JobContext(); jobContext.setJobMode(JobMode.BATCH); - FakeSource fakeSource = new FakeSource(); Config fakeSourceConfig = ConfigFactory.parseMap( Collections.singletonMap( "schema", Collections.singletonMap( "fields", ImmutableMap.of("id", "int", "name", "string")))); + FakeSource fakeSource = new FakeSource(ReadonlyConfig.fromConfig(fakeSourceConfig)); fakeSource.prepare(fakeSourceConfig); fakeSource.setJobContext(jobContext); diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/dag/TaskTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/dag/TaskTest.java index 8c677b721bac..01e105d08700 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/dag/TaskTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/dag/TaskTest.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSink; import org.apache.seatunnel.connectors.seatunnel.fake.source.FakeSource; @@ -158,13 +159,13 @@ public void testLogicalToPhysical() throws MalformedURLException { } private static FakeSource createFakeSource() { - FakeSource fakeSource = new FakeSource(); Config fakeSourceConfig = ConfigFactory.parseMap( Collections.singletonMap( "schema", Collections.singletonMap( "fields", ImmutableMap.of("id", "int", "name", "string")))); + FakeSource fakeSource = new FakeSource(ReadonlyConfig.fromConfig(fakeSourceConfig)); fakeSource.prepare(fakeSourceConfig); return fakeSource; }