diff --git a/docs/en/connector-v2/sink/OssFile.md b/docs/en/connector-v2/sink/OssFile.md index 9a6b7197253..f9e817ba562 100644 --- a/docs/en/connector-v2/sink/OssFile.md +++ b/docs/en/connector-v2/sink/OssFile.md @@ -362,6 +362,123 @@ sink { } ``` +### Multiple Table + +For extract source metadata from upstream, you can use `${database_name}`, `${table_name}` and `${schema_name}` in the path. + +```bash + +env { + parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "fake1" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + schema = { + table = "fake2" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/fake_empty/text/${table_name}" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} + +``` + ## Changelog ### 2.2.0-beta 2022-09-26 diff --git a/docs/en/connector-v2/source/OssFile.md b/docs/en/connector-v2/source/OssFile.md index d87f36bfa03..233eb76800f 100644 --- a/docs/en/connector-v2/source/OssFile.md +++ b/docs/en/connector-v2/source/OssFile.md @@ -293,6 +293,177 @@ sink { } ``` +### Multiple Table + +No need to config schema file type, eg: `orc`. + +``` +env { + parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + schema = { + table = "fake01" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + }, + { + schema = { + table = "fake02" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} +``` + +Need config schema file type, eg: `json` + +``` + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + }, + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} +``` + ## Changelog ### 2.2.0-beta 2022-09-26 diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSourceConfig.java index f46c7352840..520d40f9be2 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSourceConfig.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseFileSourceConfig.java @@ -51,15 +51,18 @@ public abstract class BaseFileSourceConfig implements Serializable { private final FileFormat fileFormat; private final ReadStrategy readStrategy; private final List filePaths; + private final ReadonlyConfig baseFileSourceConfig; public abstract HadoopConf getHadoopConfig(); public abstract String getPluginName(); public BaseFileSourceConfig(ReadonlyConfig readonlyConfig) { + this.baseFileSourceConfig = readonlyConfig; this.fileFormat = readonlyConfig.get(BaseSourceConfigOptions.FILE_FORMAT_TYPE); this.readStrategy = ReadStrategyFactory.of(readonlyConfig, getHadoopConfig()); this.filePaths = parseFilePaths(readonlyConfig); + this.catalogTable = parseCatalogTable(readonlyConfig); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/factory/BaseMultipleTableFinkSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/factory/BaseMultipleTableFinkSinkFactory.java new file mode 100644 index 00000000000..b8c60f0e197 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/factory/BaseMultipleTableFinkSinkFactory.java @@ -0,0 +1,92 @@ +/* + * 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.file.factory; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkReplaceNameConstant; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.file.sink.state.FileSinkState; + +public abstract class BaseMultipleTableFinkSinkFactory + implements TableSinkFactory< + SeaTunnelRow, FileSinkState, FileCommitInfo, FileAggregatedCommitInfo> { + + // replace the table name in sink config's path + public ReadonlyConfig generateCurrentReadonlyConfig( + ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { + // Copy the config to avoid modifying the original config + Config config = readonlyConfig.toConfig(); + + if (config.hasPath(BaseSinkConfig.FILE_PATH.key())) { + String replacedPath = + replaceCatalogTableInPath( + config.getString(BaseSinkConfig.FILE_PATH.key()), catalogTable); + config = + config.withValue( + BaseSinkConfig.FILE_PATH.key(), + ConfigValueFactory.fromAnyRef(replacedPath)); + } + + if (config.hasPath(BaseSinkConfig.TMP_PATH.key())) { + String replacedPath = + replaceCatalogTableInPath( + config.getString(BaseSinkConfig.TMP_PATH.key()), catalogTable); + config = + config.withValue( + BaseSinkConfig.TMP_PATH.key(), + ConfigValueFactory.fromAnyRef(replacedPath)); + } + + return ReadonlyConfig.fromConfig(config); + } + + public String replaceCatalogTableInPath(String originString, CatalogTable catalogTable) { + String path = originString; + TableIdentifier tableIdentifier = catalogTable.getTableId(); + if (tableIdentifier != null) { + if (tableIdentifier.getDatabaseName() != null) { + path = + path.replace( + SinkReplaceNameConstant.REPLACE_DATABASE_NAME_KEY, + tableIdentifier.getDatabaseName()); + } + if (tableIdentifier.getSchemaName() != null) { + path = + path.replace( + SinkReplaceNameConstant.REPLACE_SCHEMA_NAME_KEY, + tableIdentifier.getSchemaName()); + } + if (tableIdentifier.getTableName() != null) { + path = + path.replace( + SinkReplaceNameConstant.REPLACE_TABLE_NAME_KEY, + tableIdentifier.getTableName()); + } + } + return path; + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java index dc731879aeb..071414c9da3 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java @@ -97,7 +97,7 @@ public List getFileNamesByPath(String path) throws IOException { fileNames.addAll(getFileNamesByPath(fileStatus.getPath().toString())); continue; } - if (fileStatus.isFile() && filterFileByPattern(fileStatus)) { + if (fileStatus.isFile() && filterFileByPattern(fileStatus) && fileStatus.getLen() > 0) { // filter '_SUCCESS' file if (!fileStatus.getPath().getName().equals("_SUCCESS") && !fileStatus.getPath().getName().startsWith(".")) { diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java index d9232f4ddc5..f65a93f9095 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java @@ -17,22 +17,17 @@ package org.apache.seatunnel.connectors.seatunnel.file.local.sink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; - import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.sink.SinkReplaceNameConstant; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; 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 org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSinkConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.factory.BaseMultipleTableFinkSinkFactory; import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileAggregatedCommitInfo; import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileCommitInfo; import org.apache.seatunnel.connectors.seatunnel.file.sink.state.FileSinkState; @@ -40,9 +35,7 @@ import com.google.auto.service.AutoService; @AutoService(Factory.class) -public class LocalFileSinkFactory - implements TableSinkFactory< - SeaTunnelRow, FileSinkState, FileCommitInfo, FileAggregatedCommitInfo> { +public class LocalFileSinkFactory extends BaseMultipleTableFinkSinkFactory { @Override public String factoryIdentifier() { return FileSystemType.LOCAL.getFileSystemPluginName(); @@ -108,59 +101,4 @@ public OptionRule optionRule() { generateCurrentReadonlyConfig(readonlyConfig, catalogTable); return () -> new LocalFileSink(finalReadonlyConfig, catalogTable); } - - // replace the table name in sink config's path - private ReadonlyConfig generateCurrentReadonlyConfig( - ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { - // Copy the config to avoid modifying the original config - Config config = readonlyConfig.toConfig(); - - if (config.hasPath(BaseSinkConfig.FILE_PATH.key())) { - String replacedPath = - replaceCatalogTableInPath( - config.getString(BaseSinkConfig.FILE_PATH.key()), catalogTable); - config = - config.withValue( - BaseSinkConfig.FILE_PATH.key(), - ConfigValueFactory.fromAnyRef(replacedPath)); - } - - if (config.hasPath(BaseSinkConfig.TMP_PATH.key())) { - String replacedPath = - replaceCatalogTableInPath( - config.getString(BaseSinkConfig.TMP_PATH.key()), catalogTable); - config = - config.withValue( - BaseSinkConfig.TMP_PATH.key(), - ConfigValueFactory.fromAnyRef(replacedPath)); - } - - return ReadonlyConfig.fromConfig(config); - } - - private String replaceCatalogTableInPath(String originString, CatalogTable catalogTable) { - String path = originString; - TableIdentifier tableIdentifier = catalogTable.getTableId(); - if (tableIdentifier != null) { - if (tableIdentifier.getDatabaseName() != null) { - path = - path.replace( - SinkReplaceNameConstant.REPLACE_DATABASE_NAME_KEY, - tableIdentifier.getDatabaseName()); - } - if (tableIdentifier.getSchemaName() != null) { - path = - path.replace( - SinkReplaceNameConstant.REPLACE_SCHEMA_NAME_KEY, - tableIdentifier.getSchemaName()); - } - if (tableIdentifier.getTableName() != null) { - path = - path.replace( - SinkReplaceNameConstant.REPLACE_TABLE_NAME_KEY, - tableIdentifier.getTableName()); - } - } - return path; - } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssConf.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssHadoopConf.java similarity index 69% rename from seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssConf.java rename to seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssHadoopConf.java index 730af9bfdbb..7f4c9eb8098 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssConf.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/config/OssHadoopConf.java @@ -17,15 +17,14 @@ package org.apache.seatunnel.connectors.seatunnel.file.oss.config; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; import org.apache.hadoop.fs.aliyun.oss.Constants; import java.util.HashMap; -public class OssConf extends HadoopConf { +public class OssHadoopConf extends HadoopConf { private static final String HDFS_IMPL = "org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem"; private static final String SCHEMA = "oss"; @@ -39,19 +38,16 @@ public String getSchema() { return SCHEMA; } - public OssConf(String hdfsNameKey) { + public OssHadoopConf(String hdfsNameKey) { super(hdfsNameKey); } - public static HadoopConf buildWithConfig(Config config) { - HadoopConf hadoopConf = new OssConf(config.getString(OssConfigOptions.BUCKET.key())); + public static HadoopConf buildWithConfig(ReadonlyConfig config) { + HadoopConf hadoopConf = new OssHadoopConf(config.get(OssConfigOptions.BUCKET)); HashMap ossOptions = new HashMap<>(); - ossOptions.put( - Constants.ACCESS_KEY_ID, config.getString(OssConfigOptions.ACCESS_KEY.key())); - ossOptions.put( - Constants.ACCESS_KEY_SECRET, - config.getString(OssConfigOptions.ACCESS_SECRET.key())); - ossOptions.put(Constants.ENDPOINT_KEY, config.getString(OssConfigOptions.ENDPOINT.key())); + ossOptions.put(Constants.ACCESS_KEY_ID, config.get(OssConfigOptions.ACCESS_KEY)); + ossOptions.put(Constants.ACCESS_KEY_SECRET, config.get(OssConfigOptions.ACCESS_SECRET)); + ossOptions.put(Constants.ENDPOINT_KEY, config.get(OssConfigOptions.ENDPOINT)); hadoopConf.setExtraOptions(ossOptions); return hadoopConf; } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSink.java index 4b32e93c397..5cab55f8edd 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSink.java @@ -17,47 +17,23 @@ package org.apache.seatunnel.connectors.seatunnel.file.oss.sink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.common.config.CheckConfigUtil; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; -import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; -import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssConf; -import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssConfigOptions; -import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseFileSink; +import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssHadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseMultipleTableFileSink; import com.google.auto.service.AutoService; @AutoService(SeaTunnelSink.class) -public class OssFileSink extends BaseFileSink { - @Override - public String getPluginName() { - return FileSystemType.OSS.getFileSystemPluginName(); +public class OssFileSink extends BaseMultipleTableFileSink { + public OssFileSink(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { + super(OssHadoopConf.buildWithConfig(readonlyConfig), readonlyConfig, catalogTable); } @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - super.prepare(pluginConfig); - CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, - OssConfigOptions.FILE_PATH.key(), - OssConfigOptions.ENDPOINT.key(), - OssConfigOptions.ACCESS_KEY.key(), - OssConfigOptions.ACCESS_SECRET.key(), - OssConfigOptions.BUCKET.key()); - if (!result.isSuccess()) { - throw new FileConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SINK, result.getMsg())); - } - hadoopConf = OssConf.buildWithConfig(pluginConfig); + public String getPluginName() { + return FileSystemType.OSS.getFileSystemPluginName(); } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java index 6f168924c8e..49b5ff8bfa4 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java @@ -17,23 +17,37 @@ package org.apache.seatunnel.connectors.seatunnel.file.oss.sink; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +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 org.apache.seatunnel.connectors.seatunnel.file.config.BaseSinkConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.factory.BaseMultipleTableFinkSinkFactory; import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssConfigOptions; import com.google.auto.service.AutoService; @AutoService(Factory.class) -public class OssFileSinkFactory implements TableSinkFactory { +public class OssFileSinkFactory extends BaseMultipleTableFinkSinkFactory { @Override public String factoryIdentifier() { return FileSystemType.OSS.getFileSystemPluginName(); } + @Override + public TableSink createSink(TableSinkFactoryContext context) { + ReadonlyConfig readonlyConfig = context.getOptions(); + CatalogTable catalogTable = context.getCatalogTable(); + + ReadonlyConfig finalReadonlyConfig = + generateCurrentReadonlyConfig(readonlyConfig, catalogTable); + return () -> new OssFileSink(finalReadonlyConfig, catalogTable); + } + @Override public OptionRule optionRule() { return OptionRule.builder() 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 8a88f0a2ffe..d7222eed6dd 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 @@ -17,113 +17,19 @@ package org.apache.seatunnel.connectors.seatunnel.file.oss.source; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -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.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; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; -import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; -import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssConf; -import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssConfigOptions; -import org.apache.seatunnel.connectors.seatunnel.file.source.BaseFileSource; -import org.apache.seatunnel.connectors.seatunnel.file.source.reader.ReadStrategyFactory; +import org.apache.seatunnel.connectors.seatunnel.file.oss.source.config.MultipleTableOssFileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.source.BaseMultipleTableFileSource; -import com.google.auto.service.AutoService; +public class OssFileSource extends BaseMultipleTableFileSource { -import java.io.IOException; + public OssFileSource(ReadonlyConfig readonlyConfig) { + super(new MultipleTableOssFileSourceConfig(readonlyConfig)); + } -@AutoService(SeaTunnelSource.class) -public class OssFileSource extends BaseFileSource { @Override public String getPluginName() { return FileSystemType.OSS.getFileSystemPluginName(); } - - @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, - OssConfigOptions.FILE_PATH.key(), - OssConfigOptions.FILE_FORMAT_TYPE.key(), - OssConfigOptions.ENDPOINT.key(), - OssConfigOptions.ACCESS_KEY.key(), - OssConfigOptions.ACCESS_SECRET.key(), - OssConfigOptions.BUCKET.key()); - if (!result.isSuccess()) { - throw new FileConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SOURCE, result.getMsg())); - } - String path = pluginConfig.getString(OssConfigOptions.FILE_PATH.key()); - hadoopConf = OssConf.buildWithConfig(pluginConfig); - readStrategy = - ReadStrategyFactory.of( - pluginConfig.getString(OssConfigOptions.FILE_FORMAT_TYPE.key())); - readStrategy.setPluginConfig(pluginConfig); - readStrategy.init(hadoopConf); - try { - filePaths = readStrategy.getFileNamesByPath(path); - } catch (IOException e) { - String errorMsg = String.format("Get file list from this path [%s] failed", path); - throw new FileConnectorException( - FileConnectorErrorCode.FILE_LIST_GET_FAILED, errorMsg, e); - } - // support user-defined schema - FileFormat fileFormat = - FileFormat.valueOf( - pluginConfig - .getString(OssConfigOptions.FILE_FORMAT_TYPE.key()) - .toUpperCase()); - // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { - switch (fileFormat) { - case CSV: - case TEXT: - case JSON: - case EXCEL: - SeaTunnelRowType userDefinedSchema = - CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); - readStrategy.setSeaTunnelRowTypeInfo(userDefinedSchema); - rowType = readStrategy.getActualSeaTunnelRowTypeInfo(); - break; - case ORC: - case PARQUET: - throw new FileConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "SeaTunnel does not support user-defined schema for [parquet, orc] files"); - default: - // never got in there - throw new FileConnectorException( - CommonErrorCodeDeprecated.ILLEGAL_ARGUMENT, - "SeaTunnel does not supported this file format"); - } - } else { - if (filePaths.isEmpty()) { - // When the directory is empty, distribute default behavior schema - rowType = CatalogTableUtil.buildSimpleTextSchema(); - return; - } - try { - rowType = readStrategy.getSeaTunnelRowTypeInfo(filePaths.get(0)); - } catch (FileConnectorException e) { - String errorMsg = - String.format("Get table schema from file [%s] failed", filePaths.get(0)); - throw new FileConnectorException( - CommonErrorCodeDeprecated.TABLE_SCHEMA_GET_FAILED, errorMsg, e); - } - } - } } 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 70395abf3bd..b332d99d472 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,9 +19,12 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +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.file.config.BaseSourceConfigOptions; import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; @@ -29,6 +32,7 @@ import com.google.auto.service.AutoService; +import java.io.Serializable; import java.util.Arrays; @AutoService(Factory.class) @@ -38,15 +42,24 @@ public String factoryIdentifier() { return FileSystemType.OSS.getFileSystemPluginName(); } + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new OssFileSource(context.getOptions()); + } + @Override public OptionRule optionRule() { return OptionRule.builder() - .required(OssConfigOptions.FILE_PATH) - .required(OssConfigOptions.BUCKET) - .required(OssConfigOptions.ACCESS_KEY) - .required(OssConfigOptions.ACCESS_SECRET) - .required(OssConfigOptions.ENDPOINT) - .required(BaseSourceConfigOptions.FILE_FORMAT_TYPE) + .optional( + org.apache.seatunnel.connectors.seatunnel.file.config + .BaseSourceConfigOptions.TABLE_CONFIGS) + .optional(OssConfigOptions.FILE_PATH) + .optional(OssConfigOptions.BUCKET) + .optional(OssConfigOptions.ACCESS_KEY) + .optional(OssConfigOptions.ACCESS_SECRET) + .optional(OssConfigOptions.ENDPOINT) + .optional(BaseSourceConfigOptions.FILE_FORMAT_TYPE) .conditional( BaseSourceConfigOptions.FILE_FORMAT_TYPE, FileFormat.TEXT, diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/MultipleTableOssFileSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/MultipleTableOssFileSourceConfig.java new file mode 100644 index 00000000000..caa1b4ca70d --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/MultipleTableOssFileSourceConfig.java @@ -0,0 +1,34 @@ +/* + * 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.file.oss.source.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseFileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseMultipleTableFileSourceConfig; + +public class MultipleTableOssFileSourceConfig extends BaseMultipleTableFileSourceConfig { + + public MultipleTableOssFileSourceConfig(ReadonlyConfig ossFileSourceRootConfig) { + super(ossFileSourceRootConfig); + } + + @Override + public BaseFileSourceConfig getBaseSourceConfig(ReadonlyConfig readonlyConfig) { + return new OssFileSourceConfig(readonlyConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/OssFileSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/OssFileSourceConfig.java new file mode 100644 index 00000000000..16c3f0fa799 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/config/OssFileSourceConfig.java @@ -0,0 +1,46 @@ +/* + * 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.file.oss.source.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseFileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.oss.config.OssHadoopConf; + +import lombok.Getter; + +@Getter +public class OssFileSourceConfig extends BaseFileSourceConfig { + + private static final long serialVersionUID = 1L; + + @Override + public HadoopConf getHadoopConfig() { + return OssHadoopConf.buildWithConfig(getBaseFileSourceConfig()); + } + + @Override + public String getPluginName() { + return FileSystemType.OSS.getFileSystemPluginName(); + } + + public OssFileSourceConfig(ReadonlyConfig readonlyConfig) { + super(readonlyConfig); + } +} diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index dac8d6978bd..a97cd99aa48 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -636,12 +636,6 @@ - - org.apache.hadoop - hadoop-aliyun - ${hadoop-aliyun.version} - provided - org.apache.hadoop hadoop-aws diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/pom.xml new file mode 100644 index 00000000000..346fed24a3d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/pom.xml @@ -0,0 +1,70 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-file-oss-e2e + + 3.4.1 + + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-file-oss + ${project.version} + test + + + com.aliyun.oss + aliyun-sdk-oss + ${aliyun.sdk.oss.version} + test + + + org.apache.seatunnel + seatunnel-hadoop3-3.1.4-uber + ${project.version} + optional + test + + + org.apache.avro + avro + + + + + org.apache.seatunnel + connector-assert + ${project.version} + test + + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileIT.java new file mode 100644 index 00000000000..6ff7ae9c1e8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileIT.java @@ -0,0 +1,194 @@ +/* + * 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.file.oss; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestHelper; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import io.airlift.compress.lzo.LzopCodec; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +@Disabled("Disabled because it needs user's personal oss account to run this test") +public class OssFileIT extends TestSuiteBase { + + public static final String OSS_SDK_DOWNLOAD = + "https://repo1.maven.org/maven2/com/aliyun/oss/aliyun-sdk-oss/3.4.1/aliyun-sdk-oss-3.4.1.jar"; + public static final String JDOM_DOWNLOAD = + "https://repo1.maven.org/maven2/org/jdom/jdom/1.1/jdom-1.1.jar"; + public static final String HADOOP_ALIYUN_DOWNLOAD = + "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aliyun/3.1.4/hadoop-aliyun-3.1.4.jar"; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/oss/lib && cd /tmp/seatunnel/plugins/oss/lib && curl -O " + + OSS_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/oss/lib && curl -O " + JDOM_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/oss/lib && curl -O " + + HADOOP_ALIYUN_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + OSS_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", "-c", "cd /tmp/seatunnel/lib && curl -O " + JDOM_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + HADOOP_ALIYUN_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + /** Copy data files to oss */ + @TestTemplate + public void testOssFileReadAndWrite(TestContainer container) + throws IOException, InterruptedException { + // Copy test files to OSS + OssUtils ossUtils = new OssUtils(); + try { + ossUtils.uploadTestFiles( + "/json/e2e.json", + "test/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + true); + Path jsonLzo = convertToLzoFile(ContainerUtil.getResourcesFile("/json/e2e.json")); + ossUtils.uploadTestFiles( + jsonLzo.toString(), "test/seatunnel/read/lzo_json/e2e.json", false); + ossUtils.uploadTestFiles( + "/text/e2e.txt", + "test/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + true); + ossUtils.uploadTestFiles( + "/text/e2e_delimiter.txt", "test/seatunnel/read/text_delimiter/e2e.txt", true); + ossUtils.uploadTestFiles( + "/text/e2e_time_format.txt", + "test/seatunnel/read/text_time_format/e2e.txt", + true); + Path txtLzo = convertToLzoFile(ContainerUtil.getResourcesFile("/text/e2e.txt")); + ossUtils.uploadTestFiles( + txtLzo.toString(), "test/seatunnel/read/lzo_text/e2e.txt", false); + ossUtils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + true); + ossUtils.uploadTestFiles( + "/orc/e2e.orc", + "test/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc", + true); + ossUtils.uploadTestFiles( + "/parquet/e2e.parquet", + "test/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet", + true); + ossUtils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel_filter/name=tyrantlucifer/hobby=coding/e2e_filter.xlsx", + true); + ossUtils.createDir("tmp/fake_empty"); + } finally { + ossUtils.close(); + } + + TestHelper helper = new TestHelper(container); + + helper.execute("/excel/fake_to_oss_excel.conf"); + helper.execute("/excel/oss_excel_to_assert.conf"); + helper.execute("/excel/oss_excel_projection_to_assert.conf"); + // test write oss text file + helper.execute("/text/fake_to_oss_file_text.conf"); + helper.execute("/text/oss_file_text_lzo_to_assert.conf"); + helper.execute("/text/oss_file_delimiter_assert.conf"); + helper.execute("/text/oss_file_time_format_assert.conf"); + // test read skip header + helper.execute("/text/oss_file_text_skip_headers.conf"); + // test read oss text file + helper.execute("/text/oss_file_text_to_assert.conf"); + // test read oss text file with projection + helper.execute("/text/oss_file_text_projection_to_assert.conf"); + // test write oss json file + helper.execute("/json/fake_to_oss_file_json.conf"); + // test read oss json file + helper.execute("/json/oss_file_json_to_assert.conf"); + helper.execute("/json/oss_file_json_lzo_to_console.conf"); + // test write oss orc file + helper.execute("/orc/fake_to_oss_file_orc.conf"); + // test read oss orc file + helper.execute("/orc/oss_file_orc_to_assert.conf"); + // test read oss orc file with projection + helper.execute("/orc/oss_file_orc_projection_to_assert.conf"); + // test write oss parquet file + helper.execute("/parquet/fake_to_oss_file_parquet.conf"); + // test read oss parquet file + helper.execute("/parquet/oss_file_parquet_to_assert.conf"); + // test read oss parquet file with projection + helper.execute("/parquet/oss_file_parquet_projection_to_assert.conf"); + // test read filtered oss file + helper.execute("/excel/oss_filter_excel_to_assert.conf"); + + // test read empty directory + helper.execute("/json/oss_file_to_console.conf"); + helper.execute("/parquet/oss_file_to_console.conf"); + } + + private Path convertToLzoFile(File file) throws IOException { + LzopCodec lzo = new LzopCodec(); + Path path = Paths.get(file.getAbsolutePath() + ".lzo"); + OutputStream outputStream = lzo.createOutputStream(Files.newOutputStream(path)); + outputStream.write(Files.readAllBytes(file.toPath())); + outputStream.close(); + return path; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileWithMultipleTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileWithMultipleTableIT.java new file mode 100644 index 00000000000..39b97ce3ec8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssFileWithMultipleTableIT.java @@ -0,0 +1,162 @@ +/* + * 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.file.oss; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestHelper; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import java.io.IOException; + +@Disabled("Disabled because it needs user's personal oss account to run this test") +public class OssFileWithMultipleTableIT extends TestSuiteBase { + + public static final String OSS_SDK_DOWNLOAD = + "https://repo1.maven.org/maven2/com/aliyun/oss/aliyun-sdk-oss/3.4.1/aliyun-sdk-oss-3.4.1.jar"; + public static final String JDOM_DOWNLOAD = + "https://repo1.maven.org/maven2/org/jdom/jdom/1.1/jdom-1.1.jar"; + public static final String HADOOP_ALIYUN_DOWNLOAD = + "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aliyun/3.1.4/hadoop-aliyun-3.1.4.jar"; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/oss/lib && cd /tmp/seatunnel/plugins/oss/lib && curl -O " + + OSS_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/oss/lib && curl -O " + JDOM_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/oss/lib && curl -O " + + HADOOP_ALIYUN_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + OSS_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", "-c", "cd /tmp/seatunnel/lib && curl -O " + JDOM_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + HADOOP_ALIYUN_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + /** Copy data files to oss */ + @TestTemplate + public void addTestFiles(TestContainer container) throws IOException, InterruptedException { + // Copy test files to OSS + OssUtils ossUtils = new OssUtils(); + try { + ossUtils.uploadTestFiles( + "/json/e2e.json", + "test/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + true); + ossUtils.uploadTestFiles( + "/text/e2e.txt", + "test/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + true); + ossUtils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + true); + ossUtils.uploadTestFiles( + "/orc/e2e.orc", + "test/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc", + true); + ossUtils.uploadTestFiles( + "/parquet/e2e.parquet", + "test/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet", + true); + ossUtils.createDir("tmp/fake_empty"); + } finally { + ossUtils.close(); + } + } + + @TestTemplate + public void testFakeToOssFileInMultipleTableMode_text(TestContainer testContainer) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(testContainer); + helper.execute("/text/fake_to_oss_file_with_multiple_table.conf"); + } + + @TestTemplate + public void testOssFileReadAndWriteInMultipleTableMode_excel(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/excel/oss_excel_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testOssFileReadAndWriteInMultipleTableMode_json(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/json/oss_file_json_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testOssFileReadAndWriteInMultipleTableMode_orc(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/orc/oss_file_orc_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testOssFileReadAndWriteInMultipleTableMode_parquet(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/parquet/oss_file_parquet_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testOssFileReadAndWriteInMultipleTableMode_text(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/text/oss_file_text_to_assert_with_multipletable.conf"); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssUtils.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssUtils.java new file mode 100644 index 00000000000..0d1b90534f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/oss/OssUtils.java @@ -0,0 +1,105 @@ +/* + * 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.file.oss; + +import org.apache.seatunnel.e2e.common.util.ContainerUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.aliyun.oss.ClientException; +import com.aliyun.oss.OSS; +import com.aliyun.oss.OSSClientBuilder; +import com.aliyun.oss.OSSException; +import com.aliyun.oss.model.PutObjectResult; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; + +public class OssUtils { + private static Logger logger = LoggerFactory.getLogger(OssUtils.class); + private OSS ossClient = null; + private String endpoint = "https://oss-accelerate.aliyuncs.com"; + private String accessKeyId = "xxxxxxxxxxxxxxxxxxx"; + private String accessKeySecret = "xxxxxxxxxxxxxxxxxxx"; + private String bucket = "whale-ops"; + + public OssUtils() { + OSSClientBuilder ossClientBuilder = new OSSClientBuilder(); + ossClient = ossClientBuilder.build(endpoint, accessKeyId, accessKeySecret); + } + + public void uploadTestFiles( + String filePath, String targetFilePath, boolean isFindFromResource) { + try { + File resourcesFile = null; + if (isFindFromResource) { + resourcesFile = ContainerUtil.getResourcesFile(filePath); + } else { + resourcesFile = new File(filePath); + } + FileInputStream fileInputStream = new FileInputStream(resourcesFile); + PutObjectResult result = ossClient.putObject(bucket, targetFilePath, fileInputStream); + } catch (OSSException oe) { + logger.error( + "Caught an OSSException, which means your request made it to OSS, " + + "but was rejected with an error response for some reason."); + logger.error("Error Message:" + oe.getErrorMessage()); + logger.error("Error Code:" + oe.getErrorCode()); + logger.error("Request ID:" + oe.getRequestId()); + logger.error("Host ID:" + oe.getHostId()); + } catch (ClientException ce) { + logger.error( + "Caught an ClientException, which means the client encountered " + + "a serious internal problem while trying to communicate with OSS, " + + "such as not being able to access the network."); + logger.error("Error Message:" + ce.getMessage()); + } catch (FileNotFoundException e) { + throw new RuntimeException(e); + } + } + + public void createDir(String dir) { + try { + PutObjectResult result = + ossClient.putObject(bucket, dir, new ByteArrayInputStream("".getBytes())); + } catch (OSSException oe) { + logger.error( + "Caught an OSSException, which means your request made it to OSS, " + + "but was rejected with an error response for some reason."); + logger.error("Error Message:" + oe.getErrorMessage()); + logger.error("Error Code:" + oe.getErrorCode()); + logger.error("Request ID:" + oe.getRequestId()); + logger.error("Host ID:" + oe.getHostId()); + } catch (ClientException ce) { + logger.error( + "Caught an ClientException, which means the client encountered " + + "a serious internal problem while trying to communicate with OSS, " + + "such as not being able to access the network."); + logger.error("Error Message:" + ce.getMessage()); + } + } + + public void close() { + if (ossClient != null) { + ossClient.shutdown(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/e2e.xlsx b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/e2e.xlsx new file mode 100644 index 00000000000..87d363d7db3 Binary files /dev/null and b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/e2e.xlsx differ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/fake_to_oss_excel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/fake_to_oss_excel.conf new file mode 100644 index 00000000000..37edbef4e22 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/fake_to_oss_excel.conf @@ -0,0 +1,77 @@ +# +# 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 { + parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + OssFile { + path="/test/seatunnel/sink" + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + file_format_type = "excel" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_projection_to_assert.conf new file mode 100644 index 00000000000..99744c93bca --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_projection_to_assert.conf @@ -0,0 +1,108 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/excel" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + read_columns = [c_string, c_boolean] + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert.conf new file mode 100644 index 00000000000..477ca41b26d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert.conf @@ -0,0 +1,134 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/excel" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..19d3f375a35 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_excel_to_assert_with_multipletable.conf @@ -0,0 +1,132 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/excel" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/excel" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_filter_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_filter_excel_to_assert.conf new file mode 100644 index 00000000000..b66eccceb02 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/excel/oss_filter_excel_to_assert.conf @@ -0,0 +1,135 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/excel_filter" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + file_filter_pattern = "e2e_filter.*" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json new file mode 100644 index 00000000000..aff56314e7c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json @@ -0,0 +1,5 @@ +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json.lzo new file mode 100644 index 00000000000..e2d48d8fbc0 Binary files /dev/null and b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/e2e.json.lzo differ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/fake_to_oss_file_json.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/fake_to_oss_file_json.conf new file mode 100644 index 00000000000..4869398efab --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/fake_to_oss_file_json.conf @@ -0,0 +1,83 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/seatunnel/json" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "json" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_lzo_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_lzo_to_console.conf new file mode 100644 index 00000000000..a17fd8679b3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_lzo_to_console.conf @@ -0,0 +1,143 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + result_table_name = "fake" + path = "/test/seatunnel/read/lzo_json" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "json" + compress_codec = "lzo" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + } +} + +transform { + sql { + source_table_name = "fake" + result_table_name = "sqlresult" + query = "select * from fake where c_string = 'WArEB'" + } +} + +sink { + Assert { + source_table_name = "sqlresult" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 1 + }, + { + rule_type = MIN_ROW + rule_value = 1 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "WArEB" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_smallint + field_type = short + field_value = [ + { + equals_to = 15920 + } + ] + }, + { + field_name = c_date + field_type = date + field_value = [ + { + equals_to = "2022-04-27" + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert.conf new file mode 100644 index 00000000000..36b61de04bf --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert.conf @@ -0,0 +1,132 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..18063b90464 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_json_to_assert_with_multipletable.conf @@ -0,0 +1,128 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + }, + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_to_console.conf new file mode 100644 index 00000000000..c80a194bfbf --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/json/oss_file_to_console.conf @@ -0,0 +1,45 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/fake_empty" + file_format_type = "json" + # schema is needed for json type + schema { + + } + } +} + +sink { + Console {} +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/e2e.orc b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/e2e.orc new file mode 100644 index 00000000000..d50f6bb54dd Binary files /dev/null and b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/e2e.orc differ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/fake_to_oss_file_orc.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/fake_to_oss_file_orc.conf new file mode 100644 index 00000000000..f011bb0e1ef --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/fake_to_oss_file_orc.conf @@ -0,0 +1,84 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/seatunnel/orc" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "orc" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "zlib" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_projection_to_assert.conf new file mode 100644 index 00000000000..b5db3937db4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_projection_to_assert.conf @@ -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. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + read_columns = [c_string, c_boolean, c_double] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert.conf new file mode 100644 index 00000000000..865dc568a92 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert.conf @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..6b0a7b3b1ba --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/orc/oss_file_orc_to_assert_with_multipletable.conf @@ -0,0 +1,64 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + schema = { + table = "fake01" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + }, + { + schema = { + table = "fake02" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/orc" + file_format_type = "orc" + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/e2e.parquet b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/e2e.parquet new file mode 100644 index 00000000000..ddaad524d6a Binary files /dev/null and b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/e2e.parquet differ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/fake_to_oss_file_parquet.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/fake_to_oss_file_parquet.conf new file mode 100644 index 00000000000..4cf507dcd22 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/fake_to_oss_file_parquet.conf @@ -0,0 +1,84 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/seatunnel/parquet" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "parquet" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "gzip" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_projection_to_assert.conf new file mode 100644 index 00000000000..1c98be15bee --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_projection_to_assert.conf @@ -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. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + read_columns = [c_string, c_boolean, c_double] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert.conf new file mode 100644 index 00000000000..fbfcb6a8e83 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert.conf @@ -0,0 +1,98 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..e34806e1218 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_parquet_to_assert_with_multipletable.conf @@ -0,0 +1,64 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + schema = { + table = "fake01" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + }, + { + schema = { + table = "fake02" + } + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_to_console.conf new file mode 100644 index 00000000000..fb9e84c30d2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/parquet/oss_file_to_console.conf @@ -0,0 +1,41 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + path = "/tmp/fake_empty" + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + file_format_type = "parquet" + } +} + +sink { + Console {} +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt new file mode 100644 index 00000000000..9871cd85eb6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt @@ -0,0 +1,5 @@ +uDDrwsQQYONTNeUBIOnLAgunvDqLBObroRzdEdvDgRmgaeFyFH5456857591576298739157764687713794636442057612252MTDnafalse3313846190943192276641872220071936002.4798444E389.52375328387482E307vcIGF2023-06-0776258155390368615610.7646252373186602912023-05-08 16:08:51ipToEdierOAbwQfQzObWqiRhjkWYaMKdCbjurhstsWrAVlRyyR2905930362869031292782506910815576701385108050hArFutrue12631169122166306155952414159791708165.949173E372.1775762383875058E307kMlgO2023-05-2027214280267865241887.6424416000104182532023-10-20 03:49:02 +QIpzzZNFkLwARZDSdwdBzkegCdIRVYJnuXgxNXytAJxxaTzmDF16603816781145850255103997497062535321459349811xaTOktrue5327578191749099325840234439082792961.955231E381.5072154481920294E308GDWOu2023-05-0581449039533149712064.4515003874168475032023-07-06 22:34:11sfgxhqvOLzjdTSNcNaWfEnZqvQraSSuMPazCGhPmSrGuxggqGh111449466287130860562118177510004750271267350957FDhTstrue96247293946402921952995131535667203.3240283E384.473485404447698E307YFdwf2023-02-0429456519357128996647.9939318900994572132023-01-12 02:29:58 +xVJPgVlosBlTYSkmJCqKHMXzbZkNQKInuVMZeYGhsmzUmcLyPx137745493211075991209783701051546835517166168384qcYaifalse8318050110096656524405690917018449922.9617934E371.8901064340036343E307jaKMq2023-05-1275317114043170470995.9654034735914367862023-05-18 08:09:22raGGBnHsNwMZKemkFErUbedNjSllNcKOVUGdTpXcHGSVphHsNE86377304018502081846122308810391870441519757437JCRZStrue1829974183977114228752256792969205767.9090967E371.6286963710372255E308NBHUB2023-05-0732934086493941743464.6503746053883129532023-05-06 04:35:55 +dBgFeTKkCfnxCljyGfNEurEzCVgwpsHgmcOfYXiQHxeeQNjQuq1961913761867016982512369059615238191571813320BTfhbfalse652666522281866957533025299230722.1456136E381.2398422714159417E308YOiwg2023-10-2433001899362876139955.7235198795513055732023-06-23 13:46:46jsvmHLHlXCGFKwuqlTwAjdMckElrmqgBWvOuuKuWxcinFZWSky19959088245502706421265289671411088181469730839vUyULtrue952655754382886132164227350822215681.9033253E381.0966562906060974E308XFeKf2023-09-1731084757529957096723.2394423349193989032023-06-15 17:04:50 +obtYzIHOTKsABVtirEKEMYUYobsYlDJcFbpQUYvGxCcKlnswEG8096984004544201585383739017658796661353001394xchcntrue853141253976762312923177914159380482.8480754E381.055208146200822E308MSkTD2023-11-2420361788179232141281.9718823433892185262023-10-25 11:47:50gdCWZMGESyarjQPopBhDwKnOyDvaUDgQOEDRCmfUAagfnDDPqV8473436731118772451890654127233667151574025969ewJzLtrue6321769209768782446484076920790579202.7134378E381.1883616449174808E308STvOu2023-10-0821793351767634029460.2897683013563753232023-08-12 23:57:38 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt.lzo new file mode 100644 index 00000000000..27db8e3b592 Binary files /dev/null and b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e.txt.lzo differ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_delimiter.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_delimiter.txt new file mode 100644 index 00000000000..b87687448ca --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_delimiter.txt @@ -0,0 +1,5 @@ +qwerqwer|1972607327106509113020400507301104442513849629249|qwer|true|108|22432|11383204|723560014108175360|3.1407707E38|1.262116635132156E308|zlmzw|2023-05-25|97236477433882034782.803540569732795689|2023-03-25 04:30:13|qwerqwer1458583961104266156763552401211382922561937221393qwertrue930925142792030530244095935039344647.838737E373.3238256808030654E307Zicjq2023-10-1918739344608215707574.2737367351403166822023-10-07 08:24:27 +qwerqwer|20734545375230101131603368534223532992574063143|qwer|true|99|21567|768189694|8504422836686883840|1.3761162E38|5.460153079423635E307|dkCwG|2023-05-19|83044404421834652395.960138696348105704|2023-03-24 10:48:12|qwerqwer2774295104069855819185865051778415509162817756qwerfalse1619571127265647324402356645454202881.8446726E381.7000909191489263E308cXxQV2023-07-2713431695514477025331.5815661990272672962023-12-22 12:26:16 +qwerqwer|11147903451235598576860383707165213199232994316|qwer|true|49|21122|1110303282|2083282743100007424|1.9729736E38|1.0399541425415623E308|muvcN|2023-08-13|68941603382218317993.487441177291093700|2023-04-06 02:40:57|qwerqwer69745783829424948385550024313502468211004949206qwertrue117227855844811138143962162044856324.844609E374.992962483991954E307pPYZS2023-05-1751345924758748590630.6631664051742477762023-12-10 19:23:26 +qwerqwer|12600145717385486047323762331460409881387559257|qwer|true|54|30782|475296705|6520650210788816896|3.253564E38|1.181636072812166E308|RxBAU|2023-03-14|94882795877228509625.376060071805770292|2023-02-25 15:29:26|qwerqwer17078206571395918506189177703116985975671620089209qwerfalse11415353139002758476082670167752366081.4806856E385.82327433457546E307ppTVu2023-10-2784302780955330822761.6237458260160280852023-08-23 09:26:16 +qwerqwer|10811140972103212018816962034437650301336224152|qwer|true|82|27637|1110251085|806786601324796928|7.711023E37|4.398648945575819E307|kGVbL|2023-04-26|80164231813502964946.202647535547152674|2023-04-15 05:22:59|qwerqwer800727634149093075168463891515323059061714847070qwertrue351280654957024134756885372412119043.0538885E384.631561190310559E306leTTG2023-11-1490016690865756655359.8578360402194859042023-08-23 10:30:18 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_time_format.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_time_format.txt new file mode 100644 index 00000000000..b53cbf2e995 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/e2e_time_format.txt @@ -0,0 +1,5 @@ +PgxkWerPquZxADbwRoyZXWZYjOZGvPkcRgcvBHHlSezTHszfCM9312330451016879764123434177920993828271669125781ofduatrue02080228995824162301554221532246172.5053808E386.563348638622289E306KjJjD2023-03-2048955630047163560901.2848899427908584092023-01-11T06:25:29OxqxAMLLAWTMzSvpVKDBXwVuLuVMdhJAbNYRqEmrsQBARdHLAo98774360215016679841391554731369111688804353367rsgcotrue1212280190712202440694969264535828985.9302515E371.2125301856008725E308tVuZI2023-08-0322004483923120397310.0486453397455656992023-06-10T17:15:02 +zxMhGtbuHzxGFwmfFHIUAFvvTgUvQqetaDxOzAavJELHDSdPEV101244983376266331014538704017395315171492457270otcMntrue1024860139917168158893375714893248001.4333913E381.4334353544948444E308VdcYj2023-05-1991883965802194963022.6890574501331289452023-04-26T00:46:03PRIEJkcMnYJRsURrfhCbSgtGebklCfMXxzhZOZMudVetgtUCXc7736457412116475204164682112718260472701764785855oCRKRtrue851925389193674632146772472708622432.5017376E371.4791889801142986E308KIZKN2023-09-1334541234299674175851.0304104953008357352023-08-21T23:52:24 +EIYLFVjmjZXKcbLQtzXKMzIqLccyubcQygIssDqfcwotNQDdfH1836526392121945431313063532901700703821233811949qIlEotrue92987344051191848244308123217417652.6358307E379.12573038650651E307wrQCE2023-06-1169873404793136392100.0758355471497874132023-02-25T07:13:57IRAHziGvkRHEaUmcameBKDUCNFEjmKaafwSGblGdJGGyzQivvd12711189912021715577886030065553480147504046565RpOswfalse122122444030768933777305146193430843.3350248E381.2526133143299848E308kzyBq2023-07-1557715748983349653587.0631369056378550372023-04-28T16:02:28 +tfaoRtCwuXCoiKkBcvPOoixYBZnaUlPQMFaRjxhigVLzmBrskw190529529814451211117678789994558371211783348ccGkzfalse1112829829981778213199660821898045981.2857434E383.343575138440927E307SsSaC2023-10-2658282015679301802224.6155516408553745142023-01-26T13:15:35IETWTtUXEMkdNCiBvZPKghKHXjQUvSMaMsKYCmzsLRjFhEQXyv76798692084328842150475226014007534741586287890wbzKKtrue6625604192054124846725009551245517061.307359E381.6429413197552776E308QdOjL2023-02-2257671928068543569766.1712121225441028432023-03-28T03:01:44 +hdTngggfdRvAAMngAsZUYTEQuTFQEHdIzjOIEGIoYrTYZLIvey760974310142710026829414079475220181644270624MLIllfalse3622155133605466673524332669773532608.235333E379.308989713025347E307nrzoy2023-05-2418552644397825116718.5869443937920164442023-01-08T10:11:24bzXNzJVCPXkxeiQSpYXaVJoHWTJnKJbeIiuknfLOtQAGrKUoFr5144561031691489776106356671519647880411104465196MwxgFtrue501558654910648188780747761689955443.637149E371.4784398529023391E308cZRyO2023-06-0419268168651664178359.9430267663053671912023-11-28T19:35:41 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_text.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_text.conf new file mode 100644 index 00000000000..7378f8812d6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_text.conf @@ -0,0 +1,84 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/seatunnel/text" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_with_multiple_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_with_multiple_table.conf new file mode 100644 index 00000000000..45928001086 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/fake_to_oss_file_with_multiple_table.conf @@ -0,0 +1,125 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "fake1" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + schema = { + table = "fake2" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + } +} + +sink { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/tmp/fake_empty/text/${table_name}" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_delimiter_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_delimiter_assert.conf new file mode 100644 index 00000000000..7e412f8804d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_delimiter_assert.conf @@ -0,0 +1,108 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text_delimiter" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + file_format_type = "text" + read_columns = [c_string, c_boolean] + delimiter = "\\|" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "qwer" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + equals_to = true + } + ] + } + ] + } + } +} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_lzo_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_lzo_to_assert.conf new file mode 100644 index 00000000000..7417e74d272 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_lzo_to_assert.conf @@ -0,0 +1,142 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/lzo_text" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +transform { + sql { + source_table_name = "fake" + result_table_name = "sqlresult" + query = "select * from fake where c_string = 'MTDna'" + } +} + +sink { + Assert { + source_table_name = "sqlresult" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 1 + }, + { + rule_type = MIN_ROW + rule_value = 1 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "MTDna" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_smallint + field_type = short + field_value = [ + { + equals_to = 13846 + } + ] + }, + { + field_name = c_date + field_type = date + field_value = [ + { + equals_to = "2023-06-07" + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_projection_to_assert.conf new file mode 100644 index 00000000000..12be337d27e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_projection_to_assert.conf @@ -0,0 +1,133 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text" + file_format_type = "text" + read_columns = [c_string, c_boolean, c_double] + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_skip_headers.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_skip_headers.conf new file mode 100644 index 00000000000..263f893b7c9 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_skip_headers.conf @@ -0,0 +1,133 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text" + file_format_type = "text" + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 4 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert.conf new file mode 100644 index 00000000000..9d0415d66c6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert.conf @@ -0,0 +1,132 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..83587d7b913 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_text_to_assert_with_multipletable.conf @@ -0,0 +1,128 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + tables_configs = [ + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_time_format_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_time_format_assert.conf new file mode 100644 index 00000000000..184bb27ab33 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-oss-e2e/src/test/resources/text/oss_file_time_format_assert.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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + OssFile { + bucket = "oss://whale-ops" + access_key = "xxxxxxxxxxxxxxxxxxx" + access_secret = "xxxxxxxxxxxxxxxxxxx" + endpoint = "https://oss-accelerate.aliyuncs.com" + path = "/test/seatunnel/read/text_time_format" + file_format_type = "text" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + read_columns = [c_timestamp] + datetime_format = "yyyy-MM-dd'T'HH:mm:ss" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_timestamp + field_type = timestamp + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index a456ccc0e73..8ce365032c0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -39,6 +39,7 @@ connector-file-local-e2e connector-file-cos-e2e connector-file-sftp-e2e + connector-file-oss-e2e connector-cassandra-e2e connector-neo4j-e2e connector-http-e2e diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index 96a3901f22f..1c043866e5b 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -41,6 +41,7 @@ import org.apache.seatunnel.common.config.TypesafeConfigUtils; import org.apache.seatunnel.common.constants.CollectionConstants; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.core.starter.utils.ConfigBuilder; import org.apache.seatunnel.engine.common.config.JobConfig; @@ -275,8 +276,11 @@ private static boolean isFallback( if (e instanceof UnsupportedOperationException && "The Factory has not been implemented and the deprecated Plugin will be used." .equals(e.getMessage())) { + log.warn( + "The Factory has not been implemented and the deprecated Plugin will be used."); return true; } + log.warn(ExceptionUtils.getMessage(e)); } return false; } diff --git a/seatunnel-examples/seatunnel-engine-examples/pom.xml b/seatunnel-examples/seatunnel-engine-examples/pom.xml index 33dcaed895f..b15b7d8b4a5 100644 --- a/seatunnel-examples/seatunnel-engine-examples/pom.xml +++ b/seatunnel-examples/seatunnel-engine-examples/pom.xml @@ -38,12 +38,6 @@ ${project.version} - - org.apache.hadoop - hadoop-client - ${hadoop3.version} - - org.apache.seatunnel