From 6da7491efa3959ba6a5aa0ccf6f4ea9c66e80d0b Mon Sep 17 00:00:00 2001 From: zhangdonghao <39961809+hawk9821@users.noreply.github.com> Date: Sat, 14 Sep 2024 13:52:00 +0800 Subject: [PATCH 1/7] [Feature][Transforms-V2] LLM transforms Support custom field name (#7640) --- docs/en/transform-v2/llm.md | 9 ++- docs/zh/transform-v2/llm.md | 31 ++++---- .../seatunnel/e2e/transform/TestLLMIT.java | 8 ++ ...m_openai_transform_custom_output_name.conf | 76 +++++++++++++++++++ .../transform/nlpmodel/llm/LLMTransform.java | 10 ++- .../nlpmodel/llm/LLMTransformConfig.java | 6 ++ 6 files changed, 124 insertions(+), 16 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform_custom_output_name.conf diff --git a/docs/en/transform-v2/llm.md b/docs/en/transform-v2/llm.md index 6d036064de3..8ee5a36a9ab 100644 --- a/docs/en/transform-v2/llm.md +++ b/docs/en/transform-v2/llm.md @@ -11,11 +11,12 @@ more. ## Options | name | type | required | default value | -| ---------------------- | ------ | -------- | ------------- | +|------------------------| ------ | -------- |---------------| | model_provider | enum | yes | | | output_data_type | enum | no | String | +| output_column_name | string | no | llm_output | | prompt | string | yes | | -| inference_columns | list | no | | +| inference_columns | list | no | | | model | string | yes | | | api_key | string | yes | | | api_path | string | no | | @@ -35,6 +36,10 @@ The data type of the output data. The available options are: STRING,INT,BIGINT,DOUBLE,BOOLEAN. Default value is STRING. +### output_column_name + +Custom output data field name. A custom field name that is the same as an existing field name is replaced with 'llm_output'. + ### prompt The prompt to send to the LLM. This parameter defines how LLM will process and return data, eg: diff --git a/docs/zh/transform-v2/llm.md b/docs/zh/transform-v2/llm.md index 3ce53b78a6d..c6f7aeefead 100644 --- a/docs/zh/transform-v2/llm.md +++ b/docs/zh/transform-v2/llm.md @@ -8,19 +8,20 @@ ## 属性 -| 名称 | 类型 | 是否必须 | 默认值 | -| ---------------------- | ------ | -------- | ------ | -| model_provider | enum | yes | | -| output_data_type | enum | no | String | -| prompt | string | yes | | -| inference_columns | list | no | | -| model | string | yes | | -| api_key | string | yes | | -| api_path | string | no | | -| custom_config | map | no | | -| custom_response_parse | string | no | | -| custom_request_headers | map | no | | -| custom_request_body | map | no | | +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------------| ------ | -------- |-------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| output_column_name | string | no | llm_output | +| prompt | string | yes | | +| inference_columns | list | no | | +| model | string | yes | | +| api_key | string | yes | | +| api_path | string | no | | +| custom_config | map | no | | +| custom_response_parse | string | no | | +| custom_request_headers | map | no | | +| custom_request_body | map | no | | ### model_provider @@ -33,6 +34,10 @@ OPENAI、DOUBAO、KIMIAI、CUSTOM STRING,INT,BIGINT,DOUBLE,BOOLEAN. 默认值为 STRING。 +### output_column_name + +自定义输出数据字段名称。自定义字段名称与现有字段名称相同时,将替换为`llm_output`。 + ### prompt 发送到 LLM 的提示。此参数定义 LLM 将如何处理和返回数据,例如: diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java index b97d7182e10..d98a5e7e333 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java @@ -104,6 +104,14 @@ public void testLLMWithOpenAIColumns(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode()); } + @TestTemplate + public void testLLMWithOpenAIOutputColumnName(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = + container.executeJob("/llm_openai_transform_custom_output_name.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + @TestTemplate public void testLLMWithCustomModel(TestContainer container) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform_custom_output_name.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform_custom_output_name.conf new file mode 100644 index 00000000000..c3d17dc4230 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform_custom_output_name.conf @@ -0,0 +1,76 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + LLM { + source_table_name = "fake" + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + output_column_name = "nationality" + prompt = "Determine whether someone is Chinese or American by their name" + openai.api_path = "http://mockserver:1080/v1/chat/completions" + result_table_name = "llm_output" + } +} + +sink { + Assert { + source_table_name = "llm_output" + rules = + { + field_rules = [ + { + field_name = "nationality" + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index a29fd677cad..08ae42e4436 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java @@ -36,6 +36,7 @@ import lombok.NonNull; import lombok.SneakyThrows; +import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -150,8 +151,15 @@ protected Object getOutputFieldValue(SeaTunnelRowAccessor inputRow) { @Override protected Column getOutputColumn() { + String customFieldName = config.get(LLMTransformConfig.OUTPUT_COLUMN_NAME); + String[] fieldNames = inputCatalogTable.getTableSchema().getFieldNames(); + boolean isExist = Arrays.asList(fieldNames).contains(customFieldName); + if (isExist) { + throw new IllegalArgumentException( + String.format("llm inference field name %s already exists", customFieldName)); + } return PhysicalColumn.of( - "llm_output", outputDataType, (Long) null, true, null, "Output column of LLM"); + customFieldName, outputDataType, (Long) null, true, null, "Output column of LLM"); } @SneakyThrows diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java index c45bfb8f396..b26e4791ce9 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java @@ -37,6 +37,12 @@ public class LLMTransformConfig extends ModelTransformConfig { .noDefaultValue() .withDescription("The row projection field of each inference"); + public static final Option OUTPUT_COLUMN_NAME = + Options.key("output_column_name") + .stringType() + .defaultValue("llm_output") + .withDescription("custom field name for the llm output data"); + public static final Option INFERENCE_BATCH_SIZE = Options.key("inference_batch_size") .intType() From 4f812e12aeb20c301bdc5bc5fc93b47b6b2cee50 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=80=81=E7=8E=8B?= <58297137+chl-wxp@users.noreply.github.com> Date: Sat, 14 Sep 2024 21:42:45 +0800 Subject: [PATCH 2/7] [Feature][Connector-V2] Ftp file sink suport multiple table and save mode (#7665) --- docs/en/connector-v2/sink/FtpFile.md | 42 +++++++ .../file/ftp/catalog/FtpFileCatalog.java | 29 +++++ .../ftp/catalog/FtpFileCatalogFactory.java | 53 +++++++++ .../seatunnel/file/ftp/config/FtpConf.java | 25 +++-- .../seatunnel/file/ftp/sink/FtpFileSink.java | 39 +------ .../file/ftp/sink/FtpFileSinkFactory.java | 25 ++++- .../file/ftp/source/FtpFileSource.java | 3 +- .../e2e/connector/file/ftp/FtpFileIT.java | 73 ++++++++++++ .../multiple_table_fake_to_ftp_file_text.conf | 105 ++++++++++++++++++ ...ultiple_table_fake_to_ftp_file_text_2.conf | 105 ++++++++++++++++++ 10 files changed, 451 insertions(+), 48 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalog.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalogFactory.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text_2.conf diff --git a/docs/en/connector-v2/sink/FtpFile.md b/docs/en/connector-v2/sink/FtpFile.md index 9305aa7e990..5b927bda126 100644 --- a/docs/en/connector-v2/sink/FtpFile.md +++ b/docs/en/connector-v2/sink/FtpFile.md @@ -64,6 +64,8 @@ By default, we use 2PC commit to ensure `exactly-once` | parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | | parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | +| data_save_mode | string | no | APPEND_DATA | Existing data processing method | ### host [string] @@ -227,6 +229,18 @@ Support writing Parquet INT96 from a 12-byte field, only valid for parquet files Only used when file_format_type is json,text,csv,xml. The encoding of the file to write. This param will be parsed by `Charset.forName(encoding)`. +### schema_save_mode [string] +Existing dir processing method. +- RECREATE_SCHEMA: will create when the dir does not exist, delete and recreate when the dir is exist +- CREATE_SCHEMA_WHEN_NOT_EXIST: will create when the dir does not exist, skipped when the dir is exist +- ERROR_WHEN_SCHEMA_NOT_EXIST: error will be reported when the dir does not exist +- IGNORE :Ignore the treatment of the table + +### data_save_mode [string] +Existing data processing method. +- DROP_DATA: preserve dir and delete data files +- APPEND_DATA: preserve dir, preserve data files +- ERROR_WHEN_DATA_EXISTS: when there is data files, an error is reported ## Example For text file format simple config @@ -273,6 +287,34 @@ FtpFile { ``` +When our source end is multiple tables, and wants different expressions to different directory, we can configure this way + +```hocon + +FtpFile { + host = "xxx.xxx.xxx.xxx" + port = 21 + user = "username" + password = "password" + path = "/data/ftp/seatunnel/job1/${table_name}" + tmp_path = "/data/ftp/seatunnel/tmp" + file_format_type = "text" + field_delimiter = "\t" + row_delimiter = "\n" + have_partition = true + partition_by = ["age"] + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + custom_filename = true + file_name_expression = "${transactionId}_${now}" + sink_columns = ["name","age"] + filename_time_format = "yyyy.MM.dd" + schema_save_mode=RECREATE_SCHEMA + data_save_mode=DROP_DATA +} + +``` + ## Changelog ### 2.2.0-beta 2022-09-26 diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalog.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalog.java new file mode 100644 index 00000000000..2bf0bf49e5e --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalog.java @@ -0,0 +1,29 @@ +/* + * 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.ftp.catalog; + +import org.apache.seatunnel.connectors.seatunnel.file.catalog.AbstractFileCatalog; +import org.apache.seatunnel.connectors.seatunnel.file.hadoop.HadoopFileSystemProxy; + +public class FtpFileCatalog extends AbstractFileCatalog { + + public FtpFileCatalog( + HadoopFileSystemProxy hadoopFileSystemProxy, String filePath, String catalogName) { + super(hadoopFileSystemProxy, filePath, catalogName); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalogFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalogFactory.java new file mode 100644 index 00000000000..74f05c12d7d --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/catalog/FtpFileCatalogFactory.java @@ -0,0 +1,53 @@ +/* + * 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.ftp.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfigOptions; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.ftp.config.FtpConf; +import org.apache.seatunnel.connectors.seatunnel.file.hadoop.HadoopFileSystemProxy; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class FtpFileCatalogFactory implements CatalogFactory { + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + HadoopFileSystemProxy fileSystemUtils = + new HadoopFileSystemProxy(FtpConf.buildWithConfig(options)); + return new FtpFileCatalog( + fileSystemUtils, + options.get(BaseSourceConfigOptions.FILE_PATH), + FileSystemType.FTP.getFileSystemPluginName()); + } + + @Override + public String factoryIdentifier() { + return FileSystemType.FTP.getFileSystemPluginName(); + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().build(); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/config/FtpConf.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/config/FtpConf.java index 9186e1d8ee9..bd98800c540 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/config/FtpConf.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/config/FtpConf.java @@ -17,18 +17,19 @@ package org.apache.seatunnel.connectors.seatunnel.file.ftp.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.seatunnel.connectors.seatunnel.file.ftp.system.FtpConnectionMode; import java.util.HashMap; +import java.util.Optional; public class FtpConf extends HadoopConf { private static final String HDFS_IMPL = "org.apache.seatunnel.connectors.seatunnel.file.ftp.system.SeaTunnelFTPFileSystem"; private static final String SCHEMA = "ftp"; - private FtpConf(String hdfsNameKey) { + public FtpConf(String hdfsNameKey) { super(hdfsNameKey); } @@ -42,20 +43,20 @@ public String getSchema() { return SCHEMA; } - public static HadoopConf buildWithConfig(Config config) { - String host = config.getString(FtpConfigOptions.FTP_HOST.key()); - int port = config.getInt(FtpConfigOptions.FTP_PORT.key()); + public static HadoopConf buildWithConfig(ReadonlyConfig config) { + String host = config.get(FtpConfigOptions.FTP_HOST); + int port = config.get(FtpConfigOptions.FTP_PORT); String defaultFS = String.format("ftp://%s:%s", host, port); HadoopConf hadoopConf = new FtpConf(defaultFS); HashMap ftpOptions = new HashMap<>(); - ftpOptions.put( - "fs.ftp.user." + host, config.getString(FtpConfigOptions.FTP_USERNAME.key())); - ftpOptions.put( - "fs.ftp.password." + host, config.getString(FtpConfigOptions.FTP_PASSWORD.key())); - if (config.hasPath(FtpConfigOptions.FTP_CONNECTION_MODE.key())) { + ftpOptions.put("fs.ftp.user." + host, config.get(FtpConfigOptions.FTP_USERNAME)); + ftpOptions.put("fs.ftp.password." + host, config.get(FtpConfigOptions.FTP_PASSWORD)); + Optional optional = + config.getOptional(FtpConfigOptions.FTP_CONNECTION_MODE); + if (optional.isPresent()) { ftpOptions.put( "fs.ftp.connection.mode", - config.getString(FtpConfigOptions.FTP_CONNECTION_MODE.key())); + config.get(FtpConfigOptions.FTP_CONNECTION_MODE).toString()); } hadoopConf.setExtraOptions(ftpOptions); return hadoopConf; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSink.java index 031d442f207..f4b271e0356 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSink.java @@ -17,46 +17,19 @@ package org.apache.seatunnel.connectors.seatunnel.file.ftp.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.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.configuration.ReadonlyConfig; +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.ftp.config.FtpConf; -import org.apache.seatunnel.connectors.seatunnel.file.ftp.config.FtpConfigOptions; -import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseFileSink; - -import com.google.auto.service.AutoService; +import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseMultipleTableFileSink; -@AutoService(SeaTunnelSink.class) -public class FtpFileSink extends BaseFileSink { +public class FtpFileSink extends BaseMultipleTableFileSink { @Override public String getPluginName() { return FileSystemType.FTP.getFileSystemPluginName(); } - @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, - FtpConfigOptions.FTP_HOST.key(), - FtpConfigOptions.FTP_PORT.key(), - FtpConfigOptions.FTP_USERNAME.key(), - FtpConfigOptions.FTP_PASSWORD.key()); - if (!result.isSuccess()) { - throw new FileConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SINK, result.getMsg())); - } - super.prepare(pluginConfig); - hadoopConf = FtpConf.buildWithConfig(pluginConfig); + public FtpFileSink(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { + super(FtpConf.buildWithConfig(readonlyConfig), readonlyConfig, catalogTable); } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java index 24a9ed48f87..cfd2351a5c7 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/sink/FtpFileSinkFactory.java @@ -17,18 +17,27 @@ package org.apache.seatunnel.connectors.seatunnel.file.ftp.sink; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; +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.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.BaseMultipleTableFileSinkFactory; import org.apache.seatunnel.connectors.seatunnel.file.ftp.config.FtpConfigOptions; +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; import com.google.auto.service.AutoService; @AutoService(Factory.class) -public class FtpFileSinkFactory implements TableSinkFactory { +public class FtpFileSinkFactory extends BaseMultipleTableFileSinkFactory { @Override public String factoryIdentifier() { return FileSystemType.FTP.getFileSystemPluginName(); @@ -42,7 +51,11 @@ public OptionRule optionRule() { .required(FtpConfigOptions.FTP_PORT) .required(FtpConfigOptions.FTP_USERNAME) .required(FtpConfigOptions.FTP_PASSWORD) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) + .optional(BaseSinkConfig.TMP_PATH) .optional(BaseSinkConfig.FILE_FORMAT_TYPE) + .optional(BaseSinkConfig.SCHEMA_SAVE_MODE) + .optional(BaseSinkConfig.DATA_SAVE_MODE) .conditional( BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.TEXT, @@ -94,4 +107,12 @@ public OptionRule optionRule() { .optional(FtpConfigOptions.FTP_CONNECTION_MODE) .build(); } + + @Override + public TableSink + createSink(TableSinkFactoryContext context) { + ReadonlyConfig readonlyConfig = context.getOptions(); + CatalogTable catalogTable = context.getCatalogTable(); + return () -> new FtpFileSink(readonlyConfig, catalogTable); + } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java index b032717cabe..d6f0f64abb6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSource.java @@ -21,6 +21,7 @@ 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.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; @@ -78,7 +79,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "Ftp file source connector only support read [text, csv, json] files"); } String path = pluginConfig.getString(FtpConfigOptions.FILE_PATH.key()); - hadoopConf = FtpConf.buildWithConfig(pluginConfig); + hadoopConf = FtpConf.buildWithConfig(ReadonlyConfig.fromConfig(pluginConfig)); readStrategy = ReadStrategyFactory.of( pluginConfig.getString(FtpConfigOptions.FILE_FORMAT_TYPE.key())); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java index 2a1598bf32a..1b89a0bcc7c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java @@ -25,17 +25,27 @@ import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.util.ContainerUtil; +import org.apache.commons.lang3.StringUtils; + import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.com.github.dockerjava.core.command.ExecStartResultCallback; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.stream.Stream; @DisabledOnContainer( @@ -143,6 +153,69 @@ public void testFtpFileReadAndWrite(TestContainer container) helper.execute("/excel/fake_source_to_ftp_root_path_excel.conf"); } + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.FLINK}, + disabledReason = "Flink dosen't support multi-table at now") + public void testMultipleTableAndSaveMode(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + // test mult table and save_mode:RECREATE_SCHEMA DROP_DATA + String homePath = "/home/vsftpd/seatunnel"; + String path1 = "/tmp/seatunnel_mult/text/source_1"; + String path2 = "/tmp/seatunnel_mult/text/source_2"; + Assertions.assertEquals(getFileListFromContainer(homePath + path1).size(), 0); + Assertions.assertEquals(getFileListFromContainer(homePath + path2).size(), 0); + helper.execute("/text/multiple_table_fake_to_ftp_file_text.conf"); + Assertions.assertEquals(getFileListFromContainer(homePath + path1).size(), 1); + Assertions.assertEquals(getFileListFromContainer(homePath + path2).size(), 1); + helper.execute("/text/multiple_table_fake_to_ftp_file_text.conf"); + Assertions.assertEquals(getFileListFromContainer(homePath + path1).size(), 1); + Assertions.assertEquals(getFileListFromContainer(homePath + path2).size(), 1); + // test mult table and save_mode:CREATE_SCHEMA_WHEN_NOT_EXIST APPEND_DATA + String path3 = "/tmp/seatunnel_mult2/text/source_1"; + String path4 = "/tmp/seatunnel_mult2/text/source_2"; + Assertions.assertEquals(getFileListFromContainer(homePath + path3).size(), 0); + Assertions.assertEquals(getFileListFromContainer(homePath + path4).size(), 0); + helper.execute("/text/multiple_table_fake_to_ftp_file_text_2.conf"); + Assertions.assertEquals(getFileListFromContainer(homePath + path3).size(), 1); + Assertions.assertEquals(getFileListFromContainer(homePath + path4).size(), 1); + helper.execute("/text/multiple_table_fake_to_ftp_file_text_2.conf"); + Assertions.assertEquals(getFileListFromContainer(homePath + path3).size(), 2); + Assertions.assertEquals(getFileListFromContainer(homePath + path4).size(), 2); + } + + @SneakyThrows + private List getFileListFromContainer(String path) { + String command = "ls -1 " + path; + ExecCreateCmdResponse execCreateCmdResponse = + dockerClient + .execCreateCmd(ftpContainer.getContainerId()) + .withCmd("sh", "-c", command) + .withAttachStdout(true) + .withAttachStderr(true) + .exec(); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + dockerClient + .execStartCmd(execCreateCmdResponse.getId()) + .exec(new ExecStartResultCallback(outputStream, System.err)) + .awaitCompletion(); + + String output = new String(outputStream.toByteArray(), StandardCharsets.UTF_8).trim(); + List fileList = new ArrayList<>(); + log.info("container path file list is :{}", output); + String[] files = output.split("\n"); + for (String file : files) { + if (StringUtils.isNotEmpty(file)) { + log.info("container path file name is :{}", file); + fileList.add(file); + } + } + return fileList; + } + @AfterAll @Override public void tearDown() { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text.conf new file mode 100644 index 00000000000..cd28e543990 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" + + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + result_table_name = "ftp" + tables_configs = [ + { + schema = { + table = "source_1" + fields { + id = int + val_bool = boolean + val_tinyint = tinyint + val_smallint = smallint + val_int = int + val_bigint = bigint + val_float = float + val_double = double + val_decimal = "decimal(16, 1)" + val_string = string + } + } + rows = [ + { + kind = INSERT + fields = [1, true, 1, 2, 3, 4, 4.3,5.3,6.3, "NEW"] + } + ] + }, + { + schema = { + table = "source_2" + fields { + id = int + val_bool = boolean + val_tinyint = tinyint + val_smallint = smallint + val_int = int + val_bigint = bigint + val_float = float + val_double = double + val_decimal = "decimal(16, 1)" + } + } + rows = [ + { + kind = INSERT + fields = [1, true, 1, 2, 3, 4, 4.3,5.3,6.3] + } + ] + } + ] + } +} + +transform { +} + +sink { + FtpFile { + host = "ftp" + port = 21 + user = seatunnel + password = pass + path = "/tmp/seatunnel_mult/text/${table_name}" + source_table_name = "ftp" + 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" + "schema_save_mode"="RECREATE_SCHEMA" + "data_save_mode"="DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text_2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text_2.conf new file mode 100644 index 00000000000..e05a14ab86e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/text/multiple_table_fake_to_ftp_file_text_2.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" + + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + result_table_name = "ftp" + tables_configs = [ + { + schema = { + table = "source_1" + fields { + id = int + val_bool = boolean + val_tinyint = tinyint + val_smallint = smallint + val_int = int + val_bigint = bigint + val_float = float + val_double = double + val_decimal = "decimal(16, 1)" + val_string = string + } + } + rows = [ + { + kind = INSERT + fields = [1, true, 1, 2, 3, 4, 4.3,5.3,6.3, "NEW"] + } + ] + }, + { + schema = { + table = "source_2" + fields { + id = int + val_bool = boolean + val_tinyint = tinyint + val_smallint = smallint + val_int = int + val_bigint = bigint + val_float = float + val_double = double + val_decimal = "decimal(16, 1)" + } + } + rows = [ + { + kind = INSERT + fields = [1, true, 1, 2, 3, 4, 4.3,5.3,6.3] + } + ] + } + ] + } +} + +transform { +} + +sink { + FtpFile { + host = "ftp" + port = 21 + user = seatunnel + password = pass + path = "/tmp/seatunnel_mult2/text/${table_name}" + source_table_name = "ftp" + 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" + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + } +} \ No newline at end of file From 7e0c20a488a86e6a0108d05aa8f5688c7aa0ca30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=80=81=E7=8E=8B?= <58297137+chl-wxp@users.noreply.github.com> Date: Wed, 18 Sep 2024 19:20:01 +0800 Subject: [PATCH 3/7] [Feature][Connector-V2] jdbc saphana source tablepath support view and synonym (#7670) --- .../jdbc/catalog/AbstractJdbcCatalog.java | 36 ++++++ .../jdbc/catalog/saphana/SapHanaCatalog.java | 115 ++++++++++++++++++ .../connectors/seatunnel/jdbc/JdbcHanaIT.java | 46 ++++++- .../jdbc_sap_hana_test_view_and_synonym.conf | 59 +++++++++ 4 files changed, 255 insertions(+), 1 deletion(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/resources/jdbc_sap_hana_test_view_and_synonym.conf diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index e971c138930..260be79042c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -260,6 +260,14 @@ protected String getListDatabaseSql() { throw new UnsupportedOperationException(); } + protected String getListViewSql(String databaseName) { + throw new UnsupportedOperationException(); + } + + protected String getListSynonymSql(String databaseName) { + throw new UnsupportedOperationException(); + } + protected String getDatabaseWithConditionSql(String databaseName) { throw CommonError.unsupportedMethod(this.catalogName, "getDatabaseWithConditionSql"); } @@ -331,6 +339,34 @@ public List listTables(String databaseName) } } + public List listViews(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(this.catalogName, databaseName); + } + String dbUrl = getUrlFromDatabaseName(databaseName); + try { + return queryString(dbUrl, getListViewSql(databaseName), this::getTableName); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", catalogName), e); + } + } + + public List listSynonym(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(this.catalogName, databaseName); + } + String dbUrl = getUrlFromDatabaseName(databaseName); + try { + return queryString(dbUrl, getListSynonymSql(databaseName), this::getTableName); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", catalogName), e); + } + } + @Override public boolean tableExists(TablePath tablePath) throws CatalogException { String databaseName = tablePath.getDatabaseName(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java index 70b01b397e2..fce7e78eebb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java @@ -21,22 +21,35 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeMapper; +import org.apache.commons.lang3.StringUtils; + import lombok.extern.slf4j.Slf4j; import java.sql.Connection; import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Optional; import static org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeConverter.appendColumnSizeIfNeed; @@ -123,6 +136,18 @@ protected String getListTableSql(String databaseName) { "SELECT TABLE_NAME FROM TABLES WHERE SCHEMA_NAME = '%s'", databaseName); } + @Override + public String getListViewSql(String databaseName) { + return String.format( + "SELECT VIEW_NAME FROM SYS.VIEWS WHERE SCHEMA_NAME = '%s'", databaseName); + } + + @Override + public String getListSynonymSql(String databaseName) { + return String.format( + "SELECT SYNONYM_NAME FROM SYNONYMS WHERE SCHEMA_NAME = '%s'", databaseName); + } + @Override protected String getTableName(ResultSet rs) throws SQLException { return rs.getString(1); @@ -134,6 +159,96 @@ protected String getSelectColumnsSql(TablePath tablePath) { SELECT_COLUMNS_SQL_TEMPLATE, tablePath.getDatabaseName(), tablePath.getTableName()); } + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + try { + if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)) + || querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + String.format( + getListViewSql(tablePath.getDatabaseName()) + + " AND VIEW_NAME = '%s'", + tablePath.getTableName())) + || querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + String.format( + getListSynonymSql(tablePath.getDatabaseName()) + + " AND SYNONYM_NAME = '%s'", + tablePath.getSchemaAndTableName())); + } + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)); + } catch (DatabaseNotExistException e) { + return false; + } catch (SQLException e) { + throw new SeaTunnelException("Failed to querySQLResult", e); + } + } + + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(catalogName, tablePath); + } + String dbUrl; + if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { + dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + } else { + dbUrl = getUrlFromDatabaseName(defaultDatabase); + } + Connection conn = getConnection(dbUrl); + TablePath originalTablePath = tablePath; + if (listSynonym(tablePath.getDatabaseName()).contains(tablePath.getTableName())) { + String sql = + String.format( + "SELECT SYNONYM_NAME, SCHEMA_NAME, OBJECT_NAME, OBJECT_SCHEMA FROM SYNONYMS WHERE SCHEMA_NAME = '%s' AND SYNONYM_NAME = '%s' ", + tablePath.getDatabaseName(), tablePath.getTableName()); + try (PreparedStatement statement = conn.prepareStatement(sql); + final ResultSet resultSet = statement.executeQuery()) { + while (resultSet.next()) { + final String refDatabaseName = resultSet.getString("OBJECT_SCHEMA"); + final String refTableName = resultSet.getString("OBJECT_NAME"); + tablePath = TablePath.of(refDatabaseName, refTableName); + } + } catch (Exception e) { + throw new CatalogException( + String.format("Failed getting SYNONYM %s", tablePath.getFullName()), e); + } + } + try { + DatabaseMetaData metaData = conn.getMetaData(); + Optional primaryKey = getPrimaryKey(metaData, tablePath); + List constraintKeys = getConstraintKeys(metaData, tablePath); + try (PreparedStatement ps = conn.prepareStatement(getSelectColumnsSql(tablePath)); + ResultSet resultSet = ps.executeQuery()) { + + TableSchema.Builder builder = TableSchema.builder(); + buildColumnsWithErrorCheck(tablePath, resultSet, builder); + // add primary key + primaryKey.ifPresent(builder::primaryKey); + // add constraint key + constraintKeys.forEach(builder::constraintKey); + TableIdentifier tableIdentifier = getTableIdentifier(originalTablePath); + return CatalogTable.of( + tableIdentifier, + builder.build(), + buildConnectorOptions(tablePath), + Collections.emptyList(), + "", + catalogName); + } + } catch (SeaTunnelRuntimeException e) { + throw e; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed getting table %s", tablePath.getFullName()), e); + } + } + @Override protected Column buildColumn(ResultSet resultSet) throws SQLException { String columnName = resultSet.getString("COLUMN_NAME"); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHanaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHanaIT.java index d2b0667795f..49b4cb17635 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHanaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHanaIT.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeMapper; @@ -38,6 +39,7 @@ import lombok.SneakyThrows; import java.sql.Date; +import java.sql.Statement; import java.time.Duration; import java.time.LocalDate; import java.time.temporal.ChronoUnit; @@ -56,7 +58,9 @@ public class JdbcHanaIT extends AbstractJdbcIT { private static final String SOURCE_TABLE = "ALLDATATYPES"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_sap_hana_source_and_sink.conf"); + Lists.newArrayList( + "/jdbc_sap_hana_source_and_sink.conf", + "/jdbc_sap_hana_test_view_and_synonym.conf"); // TODO The current Docker image cannot handle the annotated type normally, // but the corresponding type can be handled normally on the standard HANA service @@ -214,6 +218,46 @@ protected void createSchemaIfNeeded() { } } + protected void createNeededTables() { + try (Statement statement = connection.createStatement()) { + String createTemplate = jdbcCase.getCreateSql(); + + String createSource = + String.format( + createTemplate, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), + jdbcCase.getSchema(), + jdbcCase.getSourceTable())); + statement.execute(createSource); + + if (!jdbcCase.isUseSaveModeCreateTable()) { + if (jdbcCase.getSinkCreateSql() != null) { + createTemplate = jdbcCase.getSinkCreateSql(); + } + String createSink = + String.format( + createTemplate, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), + jdbcCase.getSchema(), + jdbcCase.getSinkTable())); + statement.execute(createSink); + } + // create view and synonym + String createViewSql = + "CREATE VIEW TEST.ALLDATATYPES_VIEW AS SELECT * FROM TEST.ALLDATATYPES;"; + String createSynonymSql = + "CREATE SYNONYM TEST.ALLDATATYPES_SYNONYM FOR TEST.ALLDATATYPES;"; + statement.execute(createViewSql); + statement.execute(createSynonymSql); + connection.commit(); + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new SeaTunnelRuntimeException(JdbcITErrorCode.CREATE_TABLE_FAILED, exception); + } + } + @Override GenericContainer initContainer() { GenericContainer container = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/resources/jdbc_sap_hana_test_view_and_synonym.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/resources/jdbc_sap_hana_test_view_and_synonym.conf new file mode 100644 index 00000000000..aec552c208c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/resources/jdbc_sap_hana_test_view_and_synonym.conf @@ -0,0 +1,59 @@ +# +# 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 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:sap://e2e_saphana:39017" + driver = "com.sap.db.jdbc.Driver" + connection_check_timeout_sec = 1000 + user = "SYSTEM" + password = "testPassword123" + "table_list"=[ + { + "table_path"="TEST.ALLDATATYPES_VIEW" + }, + { + "table_path"="TEST.ALLDATATYPES_SYNONYM" + } + ] + } + +} + +transform { +} + +sink { + Jdbc { + url = "jdbc:sap://e2e_saphana:39017" + driver = "com.sap.db.jdbc.Driver" + connection_check_timeout_sec = 1000 + user = "SYSTEM" + password = "testPassword123" + database = "TEST" + table = "${table_name}_sink" + generate_sink_sql = true + schema_save_mode = RECREATE_SCHEMA + data_save_mode = DROP_DATA + } +} + From 6765312357fcd3d751cb4e6cfebd3cedeed5f1f5 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 18 Sep 2024 19:28:42 +0800 Subject: [PATCH 4/7] [Improve][Document] Refactor job env config document (#7631) --- docs/en/concept/JobEnvConfig.md | 22 ++++++++++++++-------- docs/en/connector-v2/sink/Assert.md | 2 +- docs/en/start-v2/docker/docker.md | 12 ++++++------ docs/en/start-v2/locally/deployment.md | 2 +- docs/zh/concept/JobEnvConfig.md | 22 ++++++++++++++-------- docs/zh/start-v2/docker/docker.md | 12 ++++++------ docs/zh/start-v2/locally/deployment.md | 2 +- 7 files changed, 43 insertions(+), 31 deletions(-) diff --git a/docs/en/concept/JobEnvConfig.md b/docs/en/concept/JobEnvConfig.md index 77c924b68f2..e21864dcebe 100644 --- a/docs/en/concept/JobEnvConfig.md +++ b/docs/en/concept/JobEnvConfig.md @@ -21,14 +21,26 @@ You can configure whether the task is in batch or stream mode through `job.mode` ### checkpoint.interval -Gets the interval in which checkpoints are periodically scheduled. +Gets the interval (milliseconds) in which checkpoints are periodically scheduled. -In `STREAMING` mode, checkpoints is required, if you do not set it, it will be obtained from the application configuration file `seatunnel.yaml`. In `BATCH` mode, you can disable checkpoints by not setting this parameter. +In `STREAMING` mode, checkpoints is required, if you do not set it, it will be obtained from the application configuration file `seatunnel.yaml`. In `BATCH` mode, you can disable checkpoints by not setting this parameter. In Zeta `STREAMING` mode, the default value is 30000 milliseconds. + +### checkpoint.timeout + +The timeout (in milliseconds) for a checkpoint. If the checkpoint is not completed before the timeout, the job will fail. In Zeta, the default value is 30000 milliseconds. ### parallelism This parameter configures the parallelism of source and sink. +### shade.identifier + +Specify the method of encryption, if you didn't have the requirement for encrypting or decrypting config files, this option can be ignored. + +For more details, you can refer to the documentation [Config Encryption Decryption](../connector-v2/Config-Encryption-Decryption.md) + +## Zeta Engine Parameter + ### job.retry.times Used to control the default retry times when a job fails. The default value is 3, and it only works in the Zeta engine. @@ -43,12 +55,6 @@ This parameter is used to specify the location of the savemode when the job is e The default value is `CLUSTER`, which means that the savemode is executed on the cluster. If you want to execute the savemode on the client, you can set it to `CLIENT`. Please use `CLUSTER` mode as much as possible, because when there are no problems with `CLUSTER` mode, we will remove `CLIENT` mode. -### shade.identifier - -Specify the method of encryption, if you didn't have the requirement for encrypting or decrypting config files, this option can be ignored. - -For more details, you can refer to the documentation [Config Encryption Decryption](../connector-v2/Config-Encryption-Decryption.md) - ## Flink Engine Parameter Here are some SeaTunnel parameter names corresponding to the names in Flink, not all of them. Please refer to the official [Flink Documentation](https://flink.apache.org/). diff --git a/docs/en/connector-v2/sink/Assert.md b/docs/en/connector-v2/sink/Assert.md index 681cb2de940..e1b93598a43 100644 --- a/docs/en/connector-v2/sink/Assert.md +++ b/docs/en/connector-v2/sink/Assert.md @@ -4,7 +4,7 @@ ## Description -A flink sink plugin which can assert illegal data by user defined rules +A sink plugin which can assert illegal data by user defined rules ## Key Features diff --git a/docs/en/start-v2/docker/docker.md b/docs/en/start-v2/docker/docker.md index 25ac52044bf..3dfe3ec8c85 100644 --- a/docs/en/start-v2/docker/docker.md +++ b/docs/en/start-v2/docker/docker.md @@ -146,14 +146,14 @@ docker run --rm -it apache/seatunnel bash -c '/bin/start-cluste there has 2 ways to create cluster within docker. -### 1. Use Docker Directly +### Use Docker Directly -1. create a network +#### create a network ```shell docker network create seatunnel-network ``` -2. start the nodes +#### start the nodes - start master node ```shell ## start master and export 5801 port @@ -213,7 +213,7 @@ docker run -d --name seatunnel_worker_1 \ ``` -### 2. Use Docker-compose +### Use Docker-compose > docker cluster mode is only support zeta engine. @@ -368,7 +368,7 @@ and run `docker-compose up -d` command, the new worker node will start, and the ### Job Operation on cluster -1. use docker as a client +#### use docker as a client - submit job : ```shell docker run --name seatunnel_client \ @@ -393,7 +393,7 @@ more command please refer [user-command](../../seatunnel-engine/user-command.md) -2. use rest api +#### use rest api please refer [Submit A Job](../../seatunnel-engine/rest-api.md#submit-a-job) diff --git a/docs/en/start-v2/locally/deployment.md b/docs/en/start-v2/locally/deployment.md index db5dbdd0fd1..8555c097f36 100644 --- a/docs/en/start-v2/locally/deployment.md +++ b/docs/en/start-v2/locally/deployment.md @@ -69,7 +69,7 @@ You can download the source code from the [download page](https://seatunnel.apac ```shell cd seatunnel -sh ./mvnw clean package -DskipTests -Dskip.spotless=true +sh ./mvnw clean install -DskipTests -Dskip.spotless=true # get the binary package cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy diff --git a/docs/zh/concept/JobEnvConfig.md b/docs/zh/concept/JobEnvConfig.md index c20797604f3..4dddb6e94a0 100644 --- a/docs/zh/concept/JobEnvConfig.md +++ b/docs/zh/concept/JobEnvConfig.md @@ -21,14 +21,26 @@ ### checkpoint.interval -获取定时调度检查点的时间间隔。 +获取定时调度检查点的时间间隔(毫秒)。 -在`STREAMING`模式下,检查点是必须的,如果不设置,将从应用程序配置文件`seatunnel.yaml`中获取。 在`BATCH`模式下,您可以通过不设置此参数来禁用检查点。 +在`STREAMING`模式下,检查点是必须的,如果不设置,将从应用程序配置文件`seatunnel.yaml`中获取。 在`BATCH`模式下,您可以通过不设置此参数来禁用检查点。在Zeta `STREAMING`模式下,默认值为30000毫秒。 + +### checkpoint.timeout + +检查点的超时时间(毫秒)。如果检查点在超时之前没有完成,作业将失败。在Zeta中,默认值为30000毫秒。 ### parallelism 该参数配置source和sink的并行度。 +### shade.identifier + +指定加密方式,如果您没有加密或解密配置文件的需求,此选项可以忽略。 + +更多详细信息,您可以参考文档 [Config Encryption Decryption](../../en/connector-v2/Config-Encryption-Decryption.md) + +## Zeta 引擎参数 + ### job.retry.times 用于控制作业失败时的默认重试次数。默认值为3,并且仅适用于Zeta引擎。 @@ -44,12 +56,6 @@ 当值为`CLIENT`时,SaveMode操作在作业提交的过程中执行,使用shell脚本提交作业时,该过程在提交作业的shell进程中执行。使用rest api提交作业时,该过程在http请求的处理线程中执行。 请尽量使用`CLUSTER`模式,因为当`CLUSTER`模式没有问题时,我们将删除`CLIENT`模式。 -### shade.identifier - -指定加密方式,如果您没有加密或解密配置文件的需求,此选项可以忽略。 - -更多详细信息,您可以参考文档 [Config Encryption Decryption](../../en/connector-v2/Config-Encryption-Decryption.md) - ## Flink 引擎参数 这里列出了一些与 Flink 中名称相对应的 SeaTunnel 参数名称,并非全部,更多内容请参考官方 [Flink Documentation](https://flink.apache.org/) for more. diff --git a/docs/zh/start-v2/docker/docker.md b/docs/zh/start-v2/docker/docker.md index bccd4b068f1..309417a5993 100644 --- a/docs/zh/start-v2/docker/docker.md +++ b/docs/zh/start-v2/docker/docker.md @@ -149,14 +149,14 @@ docker下的集群模式仅支持Zeta引擎 有两种方式来启动集群 -### 1. 直接使用Docker +### 直接使用Docker -1. 创建一个network +#### 创建一个network ```shell docker network create seatunnel-network ``` -2. 启动节点 +#### 启动节点 - 启动master节点 ```shell ## start master and export 5801 port @@ -214,7 +214,7 @@ docker run -d --name seatunnel_worker_1 \ ./bin/seatunnel-cluster.sh -r worker ``` -### 2. 使用docker-compose +### 使用docker-compose `docker-compose.yaml` 配置文件为: ```yaml version: '3.8' @@ -360,7 +360,7 @@ networks: ### 提交作业到集群 -1. 使用docker container作为客户端 +#### 使用docker container作为客户端 - 提交任务 ```shell docker run --name seatunnel_client \ @@ -383,5 +383,5 @@ docker run --name seatunnel_client \ 更多其他命令请参考[命令行工具](../../seatunnel-engine/user-command.md) -2. 使用RestAPI +#### 使用RestAPI 请参考 [提交作业](../../seatunnel-engine/rest-api.md#提交作业) \ No newline at end of file diff --git a/docs/zh/start-v2/locally/deployment.md b/docs/zh/start-v2/locally/deployment.md index e2ae8dadd03..ce17e773319 100644 --- a/docs/zh/start-v2/locally/deployment.md +++ b/docs/zh/start-v2/locally/deployment.md @@ -69,7 +69,7 @@ connector-console ```shell cd seatunnel -sh ./mvnw clean package -DskipTests -Dskip.spotless=true +sh ./mvnw clean install -DskipTests -Dskip.spotless=true # 获取构建好的二进制包 cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy From 0c0eb7e41bbe2a3003f29de1c2beaaee88bb729b Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 19 Sep 2024 13:37:03 +0800 Subject: [PATCH 5/7] [Improve][Jdbc] Jdbc truncate table should check table not database (#7654) --- .../jdbc/catalog/AbstractJdbcCatalog.java | 4 ++-- .../connectors/seatunnel/jdbc/AbstractJdbcIT.java | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index 260be79042c..772cc3bf77f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -527,11 +527,11 @@ protected void closeDatabaseConnection(String databaseName) { public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { checkNotNull(tablePath, "Table path cannot be null"); - if (!databaseExists(tablePath.getDatabaseName())) { + if (!tableExists(tablePath)) { if (ignoreIfNotExists) { return; } - throw new DatabaseNotExistException(catalogName, tablePath.getDatabaseName()); + throw new TableNotExistException(catalogName, tablePath); } truncateTableInternal(tablePath); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 6bc135dd464..6b574512c76 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; @@ -462,6 +463,19 @@ public void testCatalog() { catalog.dropDatabase(targetTablePath, false); Assertions.assertFalse(catalog.databaseExists(targetTablePath.getDatabaseName())); } + + TableNotExistException exception = + Assertions.assertThrows( + TableNotExistException.class, + () -> + catalog.truncateTable( + TablePath.of("not_exist", "not_exist", "not_exist"), + false)); + Assertions.assertEquals( + String.format( + "ErrorCode:[API-05], ErrorDescription:[Table not existed] - Table not_exist.not_exist.not_exist does not exist in Catalog %s.", + catalog.name()), + exception.getMessage()); } @Test From 0d12520f918e2e25c142a1a583cb14e9fe91c670 Mon Sep 17 00:00:00 2001 From: happyboy1024 <137260654+happyboy1024@users.noreply.github.com> Date: Thu, 19 Sep 2024 18:56:59 +0800 Subject: [PATCH 6/7] [Feature][Connector-V2] Optimize hudi sink (#7662) Co-authored-by: dengjunjie <296442618@qq.com> --- docs/en/connector-v2/sink/Hudi.md | 113 +++++- docs/zh/connector-v2/sink/Hudi.md | 156 ++++++-- .../connector-hudi/pom.xml | 17 +- .../seatunnel/hudi/catalog/HudiCatalog.java | 326 +++++++++++++++++ .../hudi/catalog/HudiCatalogFactory.java | 53 +++ .../seatunnel/hudi/config/HudiOptions.java | 91 ++--- .../seatunnel/hudi/config/HudiSinkConfig.java | 58 ++- .../hudi/config/HudiTableConfig.java | 209 +++++++++++ .../hudi/config/HudiTableOptions.java | 122 +++++++ .../hudi/exception/HudiErrorCode.java | 9 +- .../hudi/sink/HudiClientManager.java | 65 ++++ .../sink/HudiMultiTableResourceManager.java | 39 ++ .../seatunnel/hudi/sink/HudiSink.java | 83 ++++- .../seatunnel/hudi/sink/HudiSinkFactory.java | 127 +++++-- .../sink/client/HudiWriteClientProvider.java | 73 ++++ .../client/HudiWriteClientProviderProxy.java | 58 +++ .../hudi/sink/client/WriteClientProvider.java | 28 ++ .../commiter/HudiSinkAggregatedCommitter.java | 102 ++++++ .../HudiSinkAggregatedCommitter.java | 132 ------- .../AvroSchemaConverter.java | 5 +- .../sink/convert/HudiRecordConverter.java | 163 +++++++++ .../RowDataToAvroConverters.java | 6 +- .../sink/state/HudiAggregatedCommitInfo.java | 31 ++ .../hudi/sink/state/HudiCommitInfo.java | 38 ++ .../hudi/sink/state/HudiSinkState.java | 32 ++ .../hudi/sink/writer/HudiRecordWriter.java | 271 ++++++++++++++ .../hudi/sink/writer/HudiSinkWriter.java | 343 ++++-------------- .../seatunnel/hudi/util/HudiCatalogUtil.java | 37 ++ .../seatunnel/hudi/util/HudiUtil.java | 93 ++++- .../seatunnel/hudi/util/SchemaUtil.java | 147 ++++++++ .../connectors/seatunnel/hudi/HudiTest.java | 4 +- .../hudi/catalog/HudiCatalogTest.java | 175 +++++++++ .../flink/execution/SinkExecuteProcessor.java | 35 +- .../flink/execution/SinkExecuteProcessor.java | 35 +- .../spark/execution/SinkExecuteProcessor.java | 35 +- .../spark/execution/SinkExecuteProcessor.java | 35 +- .../connector-hudi-e2e/pom.xml | 28 ++ .../seatunnel/e2e/connector/hudi/HudiIT.java | 58 ++- .../e2e/connector/hudi/HudiMultiTableIT.java | 185 ++++++++++ .../hudi/HudiSeatunnelS3MultiTableIT.java | 221 +++++++++++ .../hudi/HudiSparkS3MultiTableIT.java | 202 +++++++++++ .../src/test/resources/core-site.xml | 57 +++ .../src/test/resources/fake_to_hudi.conf | 4 + .../fake_to_hudi_with_omit_config_item.conf | 52 +++ .../test/resources/multi_fake_to_hudi.conf | 93 +++++ .../src/test/resources/s3_fake_to_hudi.conf | 94 +++++ .../seatunnel/SeaTunnelContainer.java | 13 +- .../seatunnel-hadoop3-3.1.4-uber/pom.xml | 1 - 48 files changed, 3712 insertions(+), 642 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalog.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableConfig.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableOptions.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiClientManager.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiMultiTableResourceManager.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProvider.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProviderProxy.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/WriteClientProvider.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/commiter/HudiSinkAggregatedCommitter.java delete mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/committer/HudiSinkAggregatedCommitter.java rename seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/{writer => convert}/AvroSchemaConverter.java (97%) create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/HudiRecordConverter.java rename seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/{writer => convert}/RowDataToAvroConverters.java (98%) create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiAggregatedCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiSinkState.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiRecordWriter.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiCatalogUtil.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/SchemaUtil.java create mode 100644 seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiMultiTableIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSeatunnelS3MultiTableIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSparkS3MultiTableIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/core-site.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi_with_omit_config_item.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/multi_fake_to_hudi.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/s3_fake_to_hudi.conf diff --git a/docs/en/connector-v2/sink/Hudi.md b/docs/en/connector-v2/sink/Hudi.md index 5922b604a13..6c424fde15e 100644 --- a/docs/en/connector-v2/sink/Hudi.md +++ b/docs/en/connector-v2/sink/Hudi.md @@ -14,33 +14,74 @@ Used to write data to Hudi. ## Options -| name | type | required | default value | +Base configuration: + +| name | type | required | default value | +|----------------------------|---------|----------|-----------------------------| +| table_dfs_path | string | yes | - | +| conf_files_path | string | no | - | +| table_list | Array | no | - | +| auto_commit | boolean | no | true | +| schema_save_mode | enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST| +| common-options | Config | no | - | + +Table list configuration: + +| name | type | required | default value | |----------------------------|--------|----------|---------------| | table_name | string | yes | - | -| table_dfs_path | string | yes | - | -| conf_files_path | string | no | - | +| database | string | no | default | +| table_type | enum | no | COPY_ON_WRITE | +| op_type | enum | no | insert | | record_key_fields | string | no | - | | partition_fields | string | no | - | -| table_type | enum | no | copy_on_write | -| op_type | enum | no | insert | | batch_interval_ms | Int | no | 1000 | +| batch_size | Int | no | 1000 | | insert_shuffle_parallelism | Int | no | 2 | | upsert_shuffle_parallelism | Int | no | 2 | | min_commits_to_keep | Int | no | 20 | | max_commits_to_keep | Int | no | 30 | -| common-options | config | no | - | +| index_type | enum | no | BLOOM | +| index_class_name | string | no | - | +| record_byte_size | Int | no | 1024 | + +Note: When this configuration corresponds to a single table, you can flatten the configuration items in table_list to the outer layer. ### table_name [string] `table_name` The name of hudi table. +### database [string] + +`database` The database of hudi table. + ### table_dfs_path [string] -`table_dfs_path` The dfs root path of hudi table,such as 'hdfs://nameserivce/data/hudi/hudi_table/'. +`table_dfs_path` The dfs root path of hudi table, such as 'hdfs://nameserivce/data/hudi/'. ### table_type [enum] -`table_type` The type of hudi table. The value is 'copy_on_write' or 'merge_on_read'. +`table_type` The type of hudi table. The value is `COPY_ON_WRITE` or `MERGE_ON_READ`. + +### record_key_fields [string] + +`record_key_fields` The record key fields of hudi table, its are used to generate record key. It must be configured when op_type is `UPSERT`. + +### partition_fields [string] + +`partition_fields` The partition key fields of hudi table, its are used to generate partition. + +### index_type [string] + +`index_type` The index type of hudi table. Currently, `BLOOM`, `SIMPLE`, and `GLOBAL SIMPLE` are supported. + +### index_class_name [string] + +`index_class_name` The customized index classpath of hudi table, example `org.apache.seatunnel.connectors.seatunnel.hudi.index.CustomHudiIndex`. + +### record_byte_size [Int] + +`record_byte_size` The byte size of each record, This value can be used to help calculate the approximate number of records in each hudi data file. Adjusting this value can effectively reduce the number of hudi data file write magnifications. ### conf_files_path [string] @@ -48,12 +89,16 @@ Used to write data to Hudi. ### op_type [enum] -`op_type` The operation type of hudi table. The value is 'insert' or 'upsert' or 'bulk_insert'. +`op_type` The operation type of hudi table. The value is `insert` or `upsert` or `bulk_insert`. ### batch_interval_ms [Int] `batch_interval_ms` The interval time of batch write to hudi table. +### batch_size [Int] + +`batch_size` The size of batch write to hudi table. + ### insert_shuffle_parallelism [Int] `insert_shuffle_parallelism` The parallelism of insert data to hudi table. @@ -70,19 +115,35 @@ Used to write data to Hudi. `max_commits_to_keep` The max commits to keep of hudi table. +### auto_commit [boolean] + +`auto_commit` Automatic transaction commit is enabled by default. + +### schema_save_mode [Enum] + +Before the synchronous task is turned on, different treatment schemes are selected for the existing surface structure of the target side. +Option introduction: +`RECREATE_SCHEMA` :Will create when the table does not exist, delete and rebuild when the table is saved +`CREATE_SCHEMA_WHEN_NOT_EXIST` :Will Created when the table does not exist, skipped when the table is saved +`ERROR_WHEN_SCHEMA_NOT_EXIST` :Error will be reported when the table does not exist +`IGNORE` :Ignore the treatment of the table + ### common options Source plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details. ## Examples +### single table ```hocon sink { Hudi { - table_dfs_path = "hdfs://nameserivce/data/hudi/hudi_table/" + table_dfs_path = "hdfs://nameserivce/data/" + database = "st" table_name = "test_table" - table_type = "copy_on_write" + table_type = "COPY_ON_WRITE" conf_files_path = "/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml" + batch_size = 10000 use.kerberos = true kerberos.principal = "test_user@xxx" kerberos.principal.file = "/home/test/test_user.keytab" @@ -91,9 +152,6 @@ sink { ``` ### Multiple table - -#### example1 - ```hocon env { parallelism = 1 @@ -116,9 +174,32 @@ transform { sink { Hudi { + table_dfs_path = "hdfs://nameserivce/data/" + conf_files_path = "/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml" + table_list = [ + { + database = "st1" + table_name = "role" + table_type = "COPY_ON_WRITE" + op_type="INSERT" + batch_size = 10000 + }, + { + database = "st1" + table_name = "user" + table_type = "COPY_ON_WRITE" + op_type="UPSERT" + # op_type is 'UPSERT', must configured record_key_fields + record_key_fields = "user_id" + batch_size = 10000 + }, + { + database = "st1" + table_name = "Bucket" + table_type = "MERGE_ON_READ" + } + ] ... - table_dfs_path = "hdfs://nameserivce/data/hudi/hudi_table/" - table_name = "${table_name}_test" } } ``` diff --git a/docs/zh/connector-v2/sink/Hudi.md b/docs/zh/connector-v2/sink/Hudi.md index d7beda577a9..2fbf0271358 100644 --- a/docs/zh/connector-v2/sink/Hudi.md +++ b/docs/zh/connector-v2/sink/Hudi.md @@ -10,49 +10,95 @@ - [x] [exactly-once](../../concept/connector-v2-features.md) - [x] [cdc](../../concept/connector-v2-features.md) +- [x] [support multiple table write](../../concept/connector-v2-features.md) ## 选项 -| 名称 | 类型 | 是否必需 | 默认值 | -|----------------------------|--------|------|---------------| -| table_name | string | 是 | - | -| table_dfs_path | string | 是 | - | -| conf_files_path | string | 否 | - | -| record_key_fields | string | 否 | - | -| partition_fields | string | 否 | - | -| table_type | enum | 否 | copy_on_write | -| op_type | enum | 否 | insert | -| batch_interval_ms | Int | 否 | 1000 | -| insert_shuffle_parallelism | Int | 否 | 2 | -| upsert_shuffle_parallelism | Int | 否 | 2 | -| min_commits_to_keep | Int | 否 | 20 | -| max_commits_to_keep | Int | 否 | 30 | -| common-options | config | 否 | - | +基础配置: + +| 名称 | 名称 | 是否必需 | 默认值 | +|----------------------------|--------|------ |------------------------------| +| table_dfs_path | string | 是 | - | +| conf_files_path | string | 否 | - | +| table_list | string | 否 | - | +| auto_commit | boolean| 否 | true | +| schema_save_mode | enum | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | +| common-options | config | 否 | - | + +表清单配置: + +| 名称 | 类型 | 是否必需 | 默认值 | +|----------------------------|--------|----------|---------------| +| table_name | string | yes | - | +| database | string | no | default | +| table_type | enum | no | COPY_ON_WRITE | +| op_type | enum | no | insert | +| record_key_fields | string | no | - | +| partition_fields | string | no | - | +| batch_interval_ms | Int | no | 1000 | +| batch_size | Int | no | 1000 | +| insert_shuffle_parallelism | Int | no | 2 | +| upsert_shuffle_parallelism | Int | no | 2 | +| min_commits_to_keep | Int | no | 20 | +| max_commits_to_keep | Int | no | 30 | +| index_type | enum | no | BLOOM | +| index_class_name | string | no | - | +| record_byte_size | Int | no | 1024 | + +注意: 当此配置对应于单个表时,您可以将table_list中的配置项展平到外层。 ### table_name [string] `table_name` Hudi 表的名称。 +### database [string] + +`database` Hudi 表的database. + ### table_dfs_path [string] -`table_dfs_path` Hudi 表的 DFS 根路径,例如 "hdfs://nameservice/data/hudi/hudi_table/"。 +`table_dfs_path` Hudi 表的 DFS 根路径,例如 "hdfs://nameservice/data/hudi/"。 ### table_type [enum] `table_type` Hudi 表的类型。 +### record_key_fields [string] + +`record_key_fields` Hudi 表的记录键字段, 当op_type是`UPSERT`类型时, 必须配置该项. + +### partition_fields [string] + +`partition_fields` Hudi 表的分区字段. + +### index_type [string] + +`index_type` Hudi 表的索引类型. 当前只支持`BLOOM`, `SIMPLE`, `GLOBAL SIMPLE`三种类型. + +### index_class_name [string] + +`index_class_name` Hudi 表自定义索引名称,例如: `org.apache.seatunnel.connectors.seatunnel.hudi.index.CustomHudiIndex`. + +### record_byte_size [Int] + +`record_byte_size` Hudi 表单行记录的大小, 该值可用于预估每个hudi数据文件中记录的大致数量。调整此参数与`batch_size`可以有效减少hudi数据文件写放大次数. + ### conf_files_path [string] `conf_files_path` 环境配置文件路径列表(本地路径),用于初始化 HDFS 客户端以读取 Hudi 表文件。示例:"/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml"。 ### op_type [enum] -`op_type` Hudi 表的操作类型。值可以是 'insert'、'upsert' 或 'bulk_insert'。 +`op_type` Hudi 表的操作类型。值可以是 `insert`、`upsert` 或 `bulk_insert`。 ### batch_interval_ms [Int] `batch_interval_ms` 批量写入 Hudi 表的时间间隔。 +### batch_size [Int] + +`batch_size` 批量写入 Hudi 表的记录数大小. + ### insert_shuffle_parallelism [Int] `insert_shuffle_parallelism` 插入数据到 Hudi 表的并行度。 @@ -69,24 +115,92 @@ `max_commits_to_keep` Hudi 表保留的最多提交数。 +### auto_commit [boolean] + +`auto_commit` 是否自动提交. + +### schema_save_mode [Enum] + +在启动同步任务之前,针对目标侧已有的表结构选择不同的处理方案
+选项介绍:
+`RECREATE_SCHEMA`:当表不存在时会创建,当表已存在时会删除并重建
+`CREATE_SCHEMA_WHEN_NOT_EXIST`:当表不存在时会创建,当表已存在时则跳过创建
+`ERROR_WHEN_SCHEMA_NOT_EXIST`:当表不存在时将抛出错误
+`IGNORE` :忽略对表的处理
+ ### 通用选项 数据源插件的通用参数,请参考 [Source Common Options](../sink-common-options.md) 了解详细信息。 ## 示例 +### 单表 ```hocon -source { - +sink { Hudi { - table_dfs_path = "hdfs://nameserivce/data/hudi/hudi_table/" - table_type = "cow" + table_dfs_path = "hdfs://nameserivce/data/" + database = "st" + table_name = "test_table" + table_type = "COPY_ON_WRITE" conf_files_path = "/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml" + batch_size = 10000 use.kerberos = true kerberos.principal = "test_user@xxx" kerberos.principal.file = "/home/test/test_user.keytab" } +} +``` +### 多表 +```hocon +env { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + Mysql-CDC { + base-url = "jdbc:mysql://127.0.0.1:3306/seatunnel" + username = "root" + password = "******" + + table-names = ["seatunnel.role","seatunnel.user","galileo.Bucket"] + } +} + +transform { +} + +sink { + Hudi { + table_dfs_path = "hdfs://nameserivce/data/" + conf_files_path = "/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml" + table_list = [ + { + database = "st1" + table_name = "role" + table_type = "COPY_ON_WRITE" + op_type="INSERT" + batch_size = 10000 + }, + { + database = "st1" + table_name = "user" + table_type = "COPY_ON_WRITE" + op_type="UPSERT" + # op_type is 'UPSERT', must configured record_key_fields + record_key_fields = "user_id" + batch_size = 10000 + }, + { + database = "st1" + table_name = "Bucket" + table_type = "MERGE_ON_READ" + } + ] + ... + } } ``` diff --git a/seatunnel-connectors-v2/connector-hudi/pom.xml b/seatunnel-connectors-v2/connector-hudi/pom.xml index 7bac06ca272..35fc0b0459a 100644 --- a/seatunnel-connectors-v2/connector-hudi/pom.xml +++ b/seatunnel-connectors-v2/connector-hudi/pom.xml @@ -32,9 +32,11 @@ 0.15.0 3.4 - 1.14.1 + 1.12.2 1.1.10.4 4.0.2 + 3.1.4 + connector-hudi @@ -43,12 +45,24 @@ org.apache.hudi hudi-java-client ${hudi.version} + + + org.apache.hadoop + hadoop-hdfs + + org.apache.hudi hudi-client-common ${hudi.version} + + + org.apache.hadoop + hadoop-hdfs + + @@ -87,4 +101,5 @@ + diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalog.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalog.java new file mode 100644 index 00000000000..e0a25bfd85b --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalog.java @@ -0,0 +1,326 @@ +/* + * 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.hudi.catalog; + +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; + +import org.apache.avro.Schema; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.exception.HoodieCatalogException; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.RECORD_KEY_FIELDS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.TABLE_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.convertToSchema; +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiCatalogUtil.inferTablePath; +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.SchemaUtil.convertSeaTunnelType; + +@Slf4j +public class HudiCatalog implements Catalog { + + private final String catalogName; + private final org.apache.hadoop.conf.Configuration hadoopConf; + private final String tableParentDfsPathStr; + private final Path tableParentDfsPath; + private FileSystem fs; + + public HudiCatalog(String catalogName, Configuration hadoopConf, String tableParentDfsPathStr) { + this.catalogName = catalogName; + this.hadoopConf = hadoopConf; + this.tableParentDfsPathStr = tableParentDfsPathStr; + this.tableParentDfsPath = new Path(tableParentDfsPathStr); + } + + @Override + public void open() throws CatalogException { + fs = HadoopFSUtils.getFs(tableParentDfsPathStr, hadoopConf); + try { + if (!fs.exists(tableParentDfsPath)) { + log.info("Table dfs path not exists, will be created"); + fs.mkdirs(tableParentDfsPath); + } + } catch (IOException e) { + throw new CatalogException( + String.format( + "Checking catalog path %s exists exception.", tableParentDfsPathStr), + e); + } + if (!databaseExists(getDefaultDatabase())) { + TablePath defaultDatabase = TablePath.of(getDefaultDatabase(), "default"); + createDatabase(defaultDatabase, true); + } + } + + @Override + public void close() throws CatalogException { + try { + fs.close(); + } catch (Exception e) { + log.info("Hudi catalog close error.", e); + } + } + + @Override + public String name() { + return catalogName; + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return "default"; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + if (StringUtils.isEmpty(databaseName)) { + throw new CatalogException("Database name is null or empty."); + } + return listDatabases().contains(databaseName); + } + + @Override + public List listDatabases() throws CatalogException { + try { + FileStatus[] fileStatuses = fs.listStatus(tableParentDfsPath); + return Arrays.stream(fileStatuses) + .filter(FileStatus::isDirectory) + .map(fileStatus -> fileStatus.getPath().getName()) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new CatalogException("Listing database exception.", e); + } + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(catalogName, databaseName); + } + + Path dbPath = new Path(tableParentDfsPath, databaseName); + try { + return Arrays.stream(fs.listStatus(dbPath)) + .filter(FileStatus::isDirectory) + .map(fileStatus -> fileStatus.getPath().getName()) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new CatalogException( + String.format("Listing table in database %s exception.", dbPath), e); + } + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + String basePath = inferTablePath(tableParentDfsPathStr, tablePath); + try { + return fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)) + && fs.exists( + new Path( + new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME), + HoodieTableConfig.HOODIE_PROPERTIES_FILE)); + } catch (IOException e) { + throw new CatalogException( + "Error while checking whether table exists under path:" + basePath, e); + } + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(name(), tablePath); + } + HoodieTableMetaClient hoodieTableMetaClient = + HoodieTableMetaClient.builder() + .setBasePath(inferTablePath(tableParentDfsPathStr, tablePath)) + .setConf(HadoopFSUtils.getStorageConfWithCopy(hadoopConf)) + .build(); + HoodieTableType tableType = hoodieTableMetaClient.getTableType(); + HoodieTableConfig tableConfig = hoodieTableMetaClient.getTableConfig(); + TableSchema tableSchema = convertSchema(TableSchema.builder(), tableConfig); + List partitionFields = null; + if (tableConfig.getPartitionFields().isPresent()) { + partitionFields = Arrays.asList(tableConfig.getPartitionFields().get()); + } + + Map options = new HashMap<>(); + if (tableConfig.getRecordKeyFields().isPresent()) { + options.put( + RECORD_KEY_FIELDS.key(), + String.join(",", tableConfig.getRecordKeyFields().get())); + } + options.put(TABLE_TYPE.key(), tableType.name()); + return CatalogTable.of( + TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + tableSchema, + options, + partitionFields, + null); + } + + @Override + public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "Table path cannot be null"); + checkNotNull(table, "Table cannot be null"); + + String tablePathStr = inferTablePath(tableParentDfsPathStr, tablePath); + Path path = new Path(tablePathStr); + try { + if (!fs.exists(path)) { + HoodieTableMetaClient.withPropertyBuilder() + .setTableType(table.getOptions().get(TABLE_TYPE.key())) + .setRecordKeyFields(table.getOptions().get(RECORD_KEY_FIELDS.key())) + .setTableCreateSchema( + convertToSchema(table.getSeaTunnelRowType()).toString()) + .setTableName(tablePath.getTableName()) + .setPartitionFields(String.join(",", table.getPartitionKeys())) + .setPayloadClassName(HoodieAvroPayload.class.getName()) + .initTable(new HadoopStorageConfiguration(hadoopConf), tablePathStr); + } + } catch (IOException e) { + throw new HoodieCatalogException( + String.format("Failed to create table %s", tablePath.getFullName()), e); + } + } + + @Override + public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + if (!tableExists(tablePath)) { + if (ignoreIfNotExists) { + return; + } else { + throw new TableNotExistException(catalogName, tablePath); + } + } + + Path path = new Path(inferTablePath(tableParentDfsPathStr, tablePath)); + try { + this.fs.delete(path, true); + } catch (IOException e) { + throw new CatalogException(String.format("Dropping table %s exception.", tablePath), e); + } + } + + @Override + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException("Hudi catalog not support truncate table."); + } + + @Override + public void createDatabase(TablePath tablePath, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (databaseExists(tablePath.getDatabaseName())) { + if (ignoreIfExists) { + return; + } else { + throw new DatabaseAlreadyExistException(catalogName, tablePath.getDatabaseName()); + } + } + + Path dbPath = new Path(tableParentDfsPath, tablePath.getDatabaseName()); + try { + fs.mkdirs(dbPath); + } catch (IOException e) { + throw new CatalogException( + String.format("Creating database %s exception.", tablePath.getDatabaseName()), + e); + } + } + + @Override + public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + // do nothing + if (!databaseExists(tablePath.getDatabaseName())) { + if (ignoreIfNotExists) { + return; + } else { + throw new DatabaseNotExistException(catalogName, tablePath.getDatabaseName()); + } + } + + List tables = listTables(tablePath.getDatabaseName()); + if (!tables.isEmpty()) { + throw new CatalogException( + String.format( + "Database %s not empty, can't drop it.", tablePath.getDatabaseName())); + } + + Path dbPath = new Path(tableParentDfsPath, tablePath.getDatabaseName()); + try { + fs.delete(dbPath, true); + } catch (IOException e) { + throw new CatalogException( + String.format("Dropping database %s exception.", tablePath.getDatabaseName()), + e); + } + } + + private TableSchema convertSchema( + TableSchema.Builder tableSchemaBuilder, HoodieTableConfig tableConfig) { + if (tableConfig.getTableCreateSchema().isPresent()) { + Schema schema = tableConfig.getTableCreateSchema().get(); + List fields = schema.getFields(); + for (Schema.Field field : fields) { + tableSchemaBuilder.column( + PhysicalColumn.of( + field.name(), + convertSeaTunnelType(field.name(), field.schema()), + (Long) null, + true, + null, + field.doc())); + } + } + return tableSchemaBuilder.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogFactory.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogFactory.java new file mode 100644 index 00000000000..09fc7e891b8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogFactory.java @@ -0,0 +1,53 @@ +/* + * 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.hudi.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; + +import org.apache.hadoop.conf.Configuration; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.CONF_FILES_PATH; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.TABLE_DFS_PATH; +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil.getConfiguration; + +@AutoService(Factory.class) +public class HudiCatalogFactory implements CatalogFactory { + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + Configuration hadoopConf = getConfiguration(options.get(CONF_FILES_PATH)); + String tableDfsPath = options.get(TABLE_DFS_PATH); + return new HudiCatalog(catalogName, hadoopConf, tableDfsPath); + } + + @Override + public String factoryIdentifier() { + return "Hudi"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().required(TABLE_DFS_PATH).optional(CONF_FILES_PATH).build(); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiOptions.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiOptions.java index 443d06d907a..38450e2dfdd 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiOptions.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiOptions.java @@ -17,81 +17,48 @@ package org.apache.seatunnel.connectors.seatunnel.hudi.config; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; - import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SchemaSaveMode; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.model.WriteOperationType; +import java.util.List; public interface HudiOptions { - Option CONF_FILES_PATH = - Options.key("conf_files_path") - .stringType() - .noDefaultValue() - .withDescription("hudi conf files"); - - Option TABLE_NAME = - Options.key("table_name").stringType().noDefaultValue().withDescription("table_name"); - Option TABLE_DFS_PATH = Options.key("table_dfs_path") .stringType() .noDefaultValue() - .withDescription("table_dfs_path"); + .withDescription("the dfs path of hudi table"); - Option RECORD_KEY_FIELDS = - Options.key("record_key_fields") + Option CONF_FILES_PATH = + Options.key("conf_files_path") .stringType() .noDefaultValue() - .withDescription("recordKeyFields"); + .withDescription("hudi conf files"); - Option PARTITION_FIELDS = - Options.key("partition_fields") - .stringType() + Option> TABLE_LIST = + Options.key("table_list") + .listType(HudiTableConfig.class) .noDefaultValue() - .withDescription("partitionFields"); - - Option TABLE_TYPE = - Options.key("table_type") - .type(new TypeReference() {}) - .defaultValue(HoodieTableType.COPY_ON_WRITE) - .withDescription("table_type"); - Option OP_TYPE = - Options.key("op_type") - .type(new TypeReference() {}) - .defaultValue(WriteOperationType.INSERT) - .withDescription("op_type"); - - Option BATCH_INTERVAL_MS = - Options.key("batch_interval_ms") - .intType() - .defaultValue(1000) - .withDescription("batch interval milliSecond"); - - Option INSERT_SHUFFLE_PARALLELISM = - Options.key("insert_shuffle_parallelism") - .intType() - .defaultValue(2) - .withDescription("insert_shuffle_parallelism"); - - Option UPSERT_SHUFFLE_PARALLELISM = - Options.key("upsert_shuffle_parallelism") - .intType() - .defaultValue(2) - .withDescription("upsert_shuffle_parallelism"); - - Option MIN_COMMITS_TO_KEEP = - Options.key("min_commits_to_keep") - .intType() - .defaultValue(20) - .withDescription("hoodie.keep.min.commits"); - - Option MAX_COMMITS_TO_KEEP = - Options.key("max_commits_to_keep") - .intType() - .defaultValue(30) - .withDescription("hoodie.keep.max.commits"); + .withDescription("table_list"); + + Option AUTO_COMMIT = + Options.key("auto_commit") + .booleanType() + .defaultValue(true) + .withDescription("auto commit"); + + Option SCHEMA_SAVE_MODE = + Options.key("schema_save_mode") + .enumType(SchemaSaveMode.class) + .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) + .withDescription("schema save mode"); + + Option DATA_SAVE_MODE = + Options.key("data_save_mode") + .enumType(DataSaveMode.class) + .defaultValue(DataSaveMode.APPEND_DATA) + .withDescription("data save mode"); } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSinkConfig.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSinkConfig.java index 51bd72ec618..06650e87c03 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSinkConfig.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSinkConfig.java @@ -18,14 +18,15 @@ package org.apache.seatunnel.connectors.seatunnel.hudi.config; import org.apache.seatunnel.api.configuration.ReadonlyConfig; - -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.model.WriteOperationType; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SchemaSaveMode; import lombok.Builder; import lombok.Data; import java.io.Serializable; +import java.util.List; +import java.util.Optional; @Data @Builder(builderClassName = "Builder") @@ -33,50 +34,35 @@ public class HudiSinkConfig implements Serializable { private static final long serialVersionUID = 2L; - private String tableName; - private String tableDfsPath; - private int insertShuffleParallelism; - - private int upsertShuffleParallelism; - - private int minCommitsToKeep; - - private int maxCommitsToKeep; - - private HoodieTableType tableType; - - private WriteOperationType opType; + private List tableList; private String confFilesPath; - private int batchIntervalMs; + private boolean autoCommit; - private String recordKeyFields; + private SchemaSaveMode schemaSaveMode; - private String partitionFields; + private DataSaveMode dataSaveMode; public static HudiSinkConfig of(ReadonlyConfig config) { - HudiSinkConfig.Builder builder = HudiSinkConfig.builder(); - builder.confFilesPath(config.get(HudiOptions.CONF_FILES_PATH)); - builder.tableName(config.get(HudiOptions.TABLE_NAME)); - builder.tableDfsPath(config.get(HudiOptions.TABLE_DFS_PATH)); - builder.tableType(config.get(HudiOptions.TABLE_TYPE)); - builder.opType(config.get(HudiOptions.OP_TYPE)); - - builder.batchIntervalMs(config.get(HudiOptions.BATCH_INTERVAL_MS)); - - builder.partitionFields(config.get(HudiOptions.PARTITION_FIELDS)); - - builder.recordKeyFields(config.get(HudiOptions.RECORD_KEY_FIELDS)); + Builder builder = HudiSinkConfig.builder(); + Optional optionalAutoCommit = config.getOptional(HudiOptions.AUTO_COMMIT); + Optional optionalSchemaSaveMode = + config.getOptional(HudiOptions.SCHEMA_SAVE_MODE); + Optional optionalDataSaveMode = + config.getOptional(HudiOptions.DATA_SAVE_MODE); - builder.insertShuffleParallelism(config.get(HudiOptions.INSERT_SHUFFLE_PARALLELISM)); - - builder.upsertShuffleParallelism(config.get(HudiOptions.UPSERT_SHUFFLE_PARALLELISM)); + builder.tableDfsPath(config.get(HudiOptions.TABLE_DFS_PATH)); + builder.confFilesPath(config.get(HudiOptions.CONF_FILES_PATH)); + builder.tableList(HudiTableConfig.of(config)); - builder.minCommitsToKeep(config.get(HudiOptions.MIN_COMMITS_TO_KEEP)); - builder.maxCommitsToKeep(config.get(HudiOptions.MAX_COMMITS_TO_KEEP)); + builder.autoCommit(optionalAutoCommit.orElseGet(HudiOptions.AUTO_COMMIT::defaultValue)); + builder.schemaSaveMode( + optionalSchemaSaveMode.orElseGet(HudiOptions.SCHEMA_SAVE_MODE::defaultValue)); + builder.dataSaveMode( + optionalDataSaveMode.orElseGet(HudiOptions.DATA_SAVE_MODE::defaultValue)); return builder.build(); } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableConfig.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableConfig.java new file mode 100644 index 00000000000..ba0ae33efdb --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableConfig.java @@ -0,0 +1,209 @@ +/* + * 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.hudi.config; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.seatunnel.shade.com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.index.HoodieIndex; + +import lombok.Builder; +import lombok.Data; +import lombok.experimental.Tolerate; +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.BATCH_INTERVAL_MS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.DATABASE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INDEX_CLASS_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INDEX_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INSERT_SHUFFLE_PARALLELISM; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.MAX_COMMITS_TO_KEEP; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.MIN_COMMITS_TO_KEEP; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.OP_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.PARTITION_FIELDS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.RECORD_BYTE_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.RECORD_KEY_FIELDS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.TABLE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.TABLE_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.UPSERT_SHUFFLE_PARALLELISM; + +@Data +@Builder +@JsonIgnoreProperties(ignoreUnknown = true) +@Slf4j +public class HudiTableConfig implements Serializable { + + @Tolerate + public HudiTableConfig() {} + + @JsonProperty("table_name") + private String tableName; + + @JsonProperty("database") + private String database; + + @JsonProperty("table_type") + private HoodieTableType tableType; + + @JsonProperty("op_type") + private WriteOperationType opType; + + @JsonProperty("record_key_fields") + private String recordKeyFields; + + @JsonProperty("partition_fields") + private String partitionFields; + + @JsonProperty("index_type") + private HoodieIndex.IndexType indexType; + + @JsonProperty("index_class_name") + private String indexClassName; + + @JsonProperty("record_byte_size") + private Integer recordByteSize; + + @JsonProperty("batch_size") + private int batchSize; + + @JsonProperty("batch_interval_ms") + private int batchIntervalMs; + + @JsonProperty("insert_shuffle_parallelism") + private int insertShuffleParallelism; + + @JsonProperty("upsert_shuffle_parallelism") + private int upsertShuffleParallelism; + + @JsonProperty("min_commits_to_keep") + private int minCommitsToKeep; + + @JsonProperty("max_commits_to_keep") + private int maxCommitsToKeep; + + public static List of(ReadonlyConfig connectorConfig) { + List tableList; + if (connectorConfig.getOptional(HudiOptions.TABLE_LIST).isPresent()) { + tableList = connectorConfig.get(HudiOptions.TABLE_LIST); + } else { + HudiTableConfig hudiTableConfig = + HudiTableConfig.builder() + .tableName(connectorConfig.get(TABLE_NAME)) + .database(connectorConfig.get(DATABASE)) + .tableType(connectorConfig.get(TABLE_TYPE)) + .opType(connectorConfig.get(OP_TYPE)) + .recordKeyFields(connectorConfig.get(RECORD_KEY_FIELDS)) + .partitionFields(connectorConfig.get(PARTITION_FIELDS)) + .indexType(connectorConfig.get(INDEX_TYPE)) + .indexClassName(connectorConfig.get(INDEX_CLASS_NAME)) + .recordByteSize(connectorConfig.get(RECORD_BYTE_SIZE)) + .batchIntervalMs(connectorConfig.get(BATCH_INTERVAL_MS)) + .batchSize(connectorConfig.get(BATCH_SIZE)) + .insertShuffleParallelism( + connectorConfig.get(INSERT_SHUFFLE_PARALLELISM)) + .upsertShuffleParallelism( + connectorConfig.get(UPSERT_SHUFFLE_PARALLELISM)) + .minCommitsToKeep(connectorConfig.get(MIN_COMMITS_TO_KEEP)) + .maxCommitsToKeep(connectorConfig.get(MAX_COMMITS_TO_KEEP)) + .build(); + tableList = Collections.singletonList(hudiTableConfig); + } + if (tableList.size() > 1) { + Set tableNameSet = + tableList.stream() + .map(HudiTableConfig::getTableName) + .collect(Collectors.toSet()); + if (tableNameSet.size() < tableList.size() - 1) { + throw new IllegalArgumentException( + "Please configure unique `table_name`, not allow null/duplicate table name: " + + tableNameSet); + } + } + for (HudiTableConfig hudiTableConfig : tableList) { + if (Objects.isNull(hudiTableConfig.getTableName())) { + throw new IllegalArgumentException( + "Please configure `table_name`, not allow null table name in config."); + } + if (Objects.isNull(hudiTableConfig.getDatabase())) { + log.info( + "The hudi table '{}' not set database, will uses 'default' as its database.", + hudiTableConfig.getTableName()); + hudiTableConfig.setDatabase(DATABASE.defaultValue()); + } + if (Objects.isNull(hudiTableConfig.getTableType())) { + log.info( + "The hudi table '{}' not set table type, default uses 'COPY_ON_WRITE'.", + hudiTableConfig.getTableName()); + hudiTableConfig.setTableType(HoodieTableType.COPY_ON_WRITE); + } + if (Objects.isNull(hudiTableConfig.getIndexType()) + && Objects.isNull(hudiTableConfig.getIndexClassName())) { + hudiTableConfig.setIndexType(HoodieIndex.IndexType.BLOOM); + log.info( + "The hudi table '{}' not set index type, default uses 'BLOOM'.", + hudiTableConfig.getTableName()); + } + if (Objects.isNull(hudiTableConfig.getRecordByteSize())) { + hudiTableConfig.setRecordByteSize(1024); + } + if (Objects.isNull(hudiTableConfig.getOpType())) { + hudiTableConfig.setOpType(OP_TYPE.defaultValue()); + } + if (hudiTableConfig.getBatchSize() == 0) { + hudiTableConfig.setBatchSize(BATCH_SIZE.defaultValue()); + } + if (hudiTableConfig.getBatchIntervalMs() == 0) { + hudiTableConfig.setBatchIntervalMs(BATCH_INTERVAL_MS.defaultValue()); + } + if (hudiTableConfig.getInsertShuffleParallelism() == 0) { + hudiTableConfig.setInsertShuffleParallelism( + INSERT_SHUFFLE_PARALLELISM.defaultValue()); + } + if (hudiTableConfig.getUpsertShuffleParallelism() == 0) { + hudiTableConfig.setUpsertShuffleParallelism( + UPSERT_SHUFFLE_PARALLELISM.defaultValue()); + } + if (hudiTableConfig.getMinCommitsToKeep() == 0) { + hudiTableConfig.setMinCommitsToKeep(MIN_COMMITS_TO_KEEP.defaultValue()); + } + if (hudiTableConfig.getMaxCommitsToKeep() == 0) { + hudiTableConfig.setMaxCommitsToKeep(MAX_COMMITS_TO_KEEP.defaultValue()); + } + if (Objects.isNull(hudiTableConfig.getRecordKeyFields()) + && hudiTableConfig.getOpType() == WriteOperationType.UPSERT) { + throw new IllegalArgumentException( + "Please configure `record_key_fields` of " + + hudiTableConfig.getTableName() + + ", it is necessary when the `op_type` is 'UPSERT'."); + } + } + return tableList; + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableOptions.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableOptions.java new file mode 100644 index 00000000000..e48ef7be56e --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiTableOptions.java @@ -0,0 +1,122 @@ +/* + * 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.hudi.config; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.index.HoodieIndex; + +public interface HudiTableOptions { + + Option TABLE_NAME = + Options.key("table_name") + .stringType() + .noDefaultValue() + .withDescription("hudi table name"); + + Option DATABASE = + Options.key("database") + .stringType() + .defaultValue("default") + .withDescription("hudi database name"); + + Option TABLE_TYPE = + Options.key("table_type") + .type(new TypeReference() {}) + .defaultValue(HoodieTableType.COPY_ON_WRITE) + .withDescription("hudi table type"); + + Option RECORD_KEY_FIELDS = + Options.key("record_key_fields") + .stringType() + .noDefaultValue() + .withDescription("the record key fields of hudi table"); + + Option PARTITION_FIELDS = + Options.key("partition_fields") + .stringType() + .noDefaultValue() + .withDescription("the partition fields of hudi table"); + + Option INDEX_TYPE = + Options.key("index_type") + .type(new TypeReference() {}) + .defaultValue(HoodieIndex.IndexType.BLOOM) + .withDescription( + "the index type of hudi table, currently supported: [BLOOM, SIMPLE, GLOBAL_BLOOM]"); + + Option INDEX_CLASS_NAME = + Options.key("index_class_name") + .stringType() + .noDefaultValue() + .withDescription( + "customized hudi index type, the index classpath is configured here"); + + Option RECORD_BYTE_SIZE = + Options.key("record_byte_size") + .intType() + .defaultValue(1024) + .withDescription("auto commit"); + + Option OP_TYPE = + Options.key("op_type") + .type(new TypeReference() {}) + .defaultValue(WriteOperationType.INSERT) + .withDescription("op_type"); + + Option BATCH_SIZE = + Options.key("batch_size") + .intType() + .defaultValue(1000) + .withDescription("the size of each insert batch"); + + Option BATCH_INTERVAL_MS = + Options.key("batch_interval_ms") + .intType() + .defaultValue(1000) + .withDescription("batch interval milliSecond"); + + Option INSERT_SHUFFLE_PARALLELISM = + Options.key("insert_shuffle_parallelism") + .intType() + .defaultValue(2) + .withDescription("insert_shuffle_parallelism"); + + Option UPSERT_SHUFFLE_PARALLELISM = + Options.key("upsert_shuffle_parallelism") + .intType() + .defaultValue(2) + .withDescription("upsert_shuffle_parallelism"); + + Option MIN_COMMITS_TO_KEEP = + Options.key("min_commits_to_keep") + .intType() + .defaultValue(20) + .withDescription("hoodie.keep.min.commits"); + + Option MAX_COMMITS_TO_KEEP = + Options.key("max_commits_to_keep") + .intType() + .defaultValue(30) + .withDescription("hoodie.keep.max.commits"); +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiErrorCode.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiErrorCode.java index 6c99d6486f1..11a273fdcde 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiErrorCode.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiErrorCode.java @@ -19,10 +19,15 @@ import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; -enum HudiErrorCode implements SeaTunnelErrorCode { +public enum HudiErrorCode implements SeaTunnelErrorCode { CANNOT_FIND_PARQUET_FILE( "HUDI-01", - "Hudi connector can not find parquet file in table path '', please check!"); + "Hudi connector can not find parquet file in table path '', please check!"), + FLUSH_DATA_FAILED("HUDI-02", "Flush data operation that in hudi sink connector failed"), + UNSUPPORTED_OPERATION("HUDI-03", "Unsupported operation"), + TABLE_CONFIG_NOT_FOUND("HUDI-04", "Table configuration not set."), + INITIALIZE_TABLE_FAILED("HUDI-05", "Initialize table failed"), + ; private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiClientManager.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiClientManager.java new file mode 100644 index 00000000000..7a3e691bbf8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiClientManager.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hudi.sink; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; + +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.common.model.HoodieAvroPayload; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil.createHoodieJavaWriteClient; + +@Slf4j +public class HudiClientManager { + + private final HudiSinkConfig hudiSinkConfig; + + private final Map>> + hoodieJavaWriteClientMap; + + public HudiClientManager(HudiSinkConfig hudiSinkConfig) { + this.hudiSinkConfig = hudiSinkConfig; + this.hoodieJavaWriteClientMap = new ConcurrentHashMap<>(); + } + + public HoodieJavaWriteClient getClient( + int index, String tableName, SeaTunnelRowType seaTunnelRowType) { + return hoodieJavaWriteClientMap + .computeIfAbsent(tableName, i -> new ConcurrentHashMap<>()) + .computeIfAbsent( + index, + i -> + createHoodieJavaWriteClient( + hudiSinkConfig, seaTunnelRowType, tableName)); + } + + public boolean containsClient(String tableName, int index) { + return hoodieJavaWriteClientMap.containsKey(tableName) + && hoodieJavaWriteClientMap.get(tableName).containsKey(index); + } + + public HoodieJavaWriteClient remove(String tableName, int index) { + return hoodieJavaWriteClientMap.get(tableName).get(index); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiMultiTableResourceManager.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiMultiTableResourceManager.java new file mode 100644 index 00000000000..8b2f31d93ab --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiMultiTableResourceManager.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hudi.sink; + +import org.apache.seatunnel.api.sink.MultiTableResourceManager; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Optional; + +@Slf4j +public class HudiMultiTableResourceManager implements MultiTableResourceManager { + + private final HudiClientManager clientManager; + + public HudiMultiTableResourceManager(HudiClientManager clientManager) { + this.clientManager = clientManager; + } + + @Override + public Optional getSharedResource() { + return Optional.of(clientManager); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSink.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSink.java index 4065338bbff..5bdc3b8c3ae 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSink.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSink.java @@ -17,41 +17,62 @@ package org.apache.seatunnel.connectors.seatunnel.hudi.sink; +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.Catalog; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.factory.CatalogFactory; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; -import org.apache.seatunnel.connectors.seatunnel.hudi.sink.committer.HudiSinkAggregatedCommitter; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiConnectorException; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.commiter.HudiSinkAggregatedCommitter; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiSinkState; import org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.HudiSinkWriter; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiSinkState; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; +import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory; + public class HudiSink implements SeaTunnelSink< SeaTunnelRow, HudiSinkState, HudiCommitInfo, HudiAggregatedCommitInfo>, + SupportSaveMode, SupportMultiTableSink { - private HudiSinkConfig hudiSinkConfig; - private SeaTunnelRowType seaTunnelRowType; - private CatalogTable catalogTable; + private final ReadonlyConfig config; + private final HudiSinkConfig hudiSinkConfig; + private final SeaTunnelRowType seaTunnelRowType; + private final CatalogTable catalogTable; + private final HudiTableConfig hudiTableConfig; - public HudiSink(ReadonlyConfig config, CatalogTable table) { - this.hudiSinkConfig = HudiSinkConfig.of(config); + public HudiSink( + ReadonlyConfig config, + HudiSinkConfig hudiSinkConfig, + HudiTableConfig hudiTableConfig, + CatalogTable table) { + this.config = config; + this.hudiSinkConfig = hudiSinkConfig; this.catalogTable = table; this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + this.hudiTableConfig = hudiTableConfig; } @Override @@ -60,9 +81,16 @@ public String getPluginName() { } @Override - public HudiSinkWriter restoreWriter(SinkWriter.Context context, List states) - throws IOException { - return new HudiSinkWriter(context, seaTunnelRowType, hudiSinkConfig, states); + public HudiSinkWriter createWriter(SinkWriter.Context context) throws IOException { + return new HudiSinkWriter( + context, seaTunnelRowType, hudiSinkConfig, hudiTableConfig, new ArrayList<>()); + } + + @Override + public SinkWriter restoreWriter( + SinkWriter.Context context, List states) throws IOException { + return new HudiSinkWriter( + context, seaTunnelRowType, hudiSinkConfig, hudiTableConfig, states); } @Override @@ -78,7 +106,8 @@ public Optional> getCommitInfoSerializer() { @Override public Optional> createAggregatedCommitter() throws IOException { - return Optional.of(new HudiSinkAggregatedCommitter(hudiSinkConfig, seaTunnelRowType)); + return Optional.of( + new HudiSinkAggregatedCommitter(hudiTableConfig, hudiSinkConfig, seaTunnelRowType)); } @Override @@ -87,7 +116,31 @@ public Optional> getAggregatedCommitInfoSer } @Override - public HudiSinkWriter createWriter(SinkWriter.Context context) throws IOException { - return new HudiSinkWriter(context, seaTunnelRowType, hudiSinkConfig, new ArrayList<>()); + public Optional getSaveModeHandler() { + TablePath tablePath = + TablePath.of( + catalogTable.getTableId().getDatabaseName(), + catalogTable.getTableId().getTableName()); + CatalogFactory catalogFactory = + discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + "Hudi"); + if (catalogFactory == null) { + throw new HudiConnectorException( + SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format( + "PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SINK, "Cannot find Hudi catalog factory")); + } + Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); + return Optional.of( + new DefaultSaveModeHandler( + hudiSinkConfig.getSchemaSaveMode(), + hudiSinkConfig.getDataSaveMode(), + catalog, + tablePath, + catalogTable, + null)); } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java index 7697842f826..ed21b15166a 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, @@ -21,25 +20,42 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.SinkCommonOptions; 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.connectors.seatunnel.hudi.config.HudiSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiConnectorException; + +import org.apache.commons.lang3.StringUtils; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.BATCH_INTERVAL_MS; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.AUTO_COMMIT; import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.CONF_FILES_PATH; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.INSERT_SHUFFLE_PARALLELISM; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.MAX_COMMITS_TO_KEEP; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.MIN_COMMITS_TO_KEEP; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.OP_TYPE; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.PARTITION_FIELDS; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.RECORD_KEY_FIELDS; import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.TABLE_DFS_PATH; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.TABLE_NAME; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.TABLE_TYPE; -import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.UPSERT_SHUFFLE_PARALLELISM; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiOptions.TABLE_LIST; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.BATCH_INTERVAL_MS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INDEX_CLASS_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INDEX_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.INSERT_SHUFFLE_PARALLELISM; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.MAX_COMMITS_TO_KEEP; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.MIN_COMMITS_TO_KEEP; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.OP_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.PARTITION_FIELDS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.RECORD_BYTE_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.RECORD_KEY_FIELDS; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.TABLE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.TABLE_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableOptions.UPSERT_SHUFFLE_PARALLELISM; +import static org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiErrorCode.TABLE_CONFIG_NOT_FOUND; @AutoService(Factory.class) public class HudiSinkFactory implements TableSinkFactory { @@ -51,25 +67,92 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(TABLE_DFS_PATH, TABLE_NAME) .optional( - CONF_FILES_PATH, + TABLE_NAME, + TABLE_DFS_PATH, + TABLE_TYPE, RECORD_KEY_FIELDS, PARTITION_FIELDS, - TABLE_TYPE, + INDEX_TYPE, + INDEX_CLASS_NAME, + RECORD_BYTE_SIZE, + TABLE_LIST, + CONF_FILES_PATH, OP_TYPE, + BATCH_SIZE, BATCH_INTERVAL_MS, INSERT_SHUFFLE_PARALLELISM, UPSERT_SHUFFLE_PARALLELISM, MIN_COMMITS_TO_KEEP, MAX_COMMITS_TO_KEEP, + AUTO_COMMIT, SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } @Override public TableSink createSink(TableSinkFactoryContext context) { + HudiSinkConfig hudiSinkConfig = HudiSinkConfig.of(context.getOptions()); CatalogTable catalogTable = context.getCatalogTable(); - return () -> new HudiSink(context.getOptions(), catalogTable); + HudiTableConfig hudiTableConfig = + getHudiTableConfig(hudiSinkConfig, catalogTable.getTableId().getTableName()); + TableIdentifier tableId = catalogTable.getTableId(); + + // rebuild TableIdentifier and catalogTable + TableIdentifier newTableId = + TableIdentifier.of( + tableId.getCatalogName(), + hudiTableConfig.getDatabase(), + tableId.getSchemaName(), + hudiTableConfig.getTableName()); + // partition keys + List finalPartitionKeys = catalogTable.getPartitionKeys(); + if (StringUtils.isNoneEmpty(hudiTableConfig.getPartitionFields())) { + finalPartitionKeys = Arrays.asList(hudiTableConfig.getPartitionFields().split(",")); + catalogTable + .getOptions() + .put(PARTITION_FIELDS.key(), hudiTableConfig.getPartitionFields()); + } + // record keys + if (StringUtils.isNoneEmpty(hudiTableConfig.getRecordKeyFields())) { + catalogTable + .getOptions() + .put(RECORD_KEY_FIELDS.key(), hudiTableConfig.getRecordKeyFields()); + } + // table type + catalogTable.getOptions().put(TABLE_TYPE.key(), hudiTableConfig.getTableType().name()); + catalogTable = + CatalogTable.of( + newTableId, + catalogTable.getTableSchema(), + catalogTable.getOptions(), + finalPartitionKeys, + catalogTable.getComment(), + catalogTable.getCatalogName()); + // set record keys to options + CatalogTable finalCatalogTable = catalogTable; + return () -> + new HudiSink( + context.getOptions(), hudiSinkConfig, hudiTableConfig, finalCatalogTable); + } + + private HudiTableConfig getHudiTableConfig(HudiSinkConfig hudiSinkConfig, String tableName) { + List tableList = hudiSinkConfig.getTableList(); + if (tableList.size() == 1) { + return tableList.get(0); + } else if (tableList.size() > 1) { + Optional optionalHudiTableConfig = + tableList.stream() + .filter(table -> table.getTableName().equals(tableName)) + .findFirst(); + if (!optionalHudiTableConfig.isPresent()) { + throw new HudiConnectorException( + TABLE_CONFIG_NOT_FOUND, + "The corresponding table configuration is not found"); + } + return optionalHudiTableConfig.get(); + } + throw new HudiConnectorException( + TABLE_CONFIG_NOT_FOUND, "The corresponding table configuration is not found"); } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProvider.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProvider.java new file mode 100644 index 00000000000..9437c59e38a --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProvider.java @@ -0,0 +1,73 @@ +/* + * 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.hudi.sink.client; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; + +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.common.model.HoodieAvroPayload; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil.createHoodieJavaWriteClient; + +@Slf4j +public class HudiWriteClientProvider implements WriteClientProvider, Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(HudiWriteClientProvider.class); + + private transient HoodieJavaWriteClient client; + + private final HudiSinkConfig hudiSinkConfig; + + private final String tableName; + + private final SeaTunnelRowType seaTunnelRowType; + + public HudiWriteClientProvider( + HudiSinkConfig hudiSinkConfig, String tableName, SeaTunnelRowType seaTunnelRowType) { + this.hudiSinkConfig = hudiSinkConfig; + this.tableName = tableName; + this.seaTunnelRowType = seaTunnelRowType; + } + + @Override + public HoodieJavaWriteClient getOrCreateClient() { + if (client == null) { + client = createHoodieJavaWriteClient(hudiSinkConfig, seaTunnelRowType, tableName); + } + return client; + } + + @Override + public void close() { + try { + if (client != null) { + client.close(); + } + } catch (Exception e) { + LOG.error("hudi client close failed.", e); + } + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProviderProxy.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProviderProxy.java new file mode 100644 index 00000000000..ae71edfa978 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/HudiWriteClientProviderProxy.java @@ -0,0 +1,58 @@ +/* + * 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.hudi.sink.client; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.HudiClientManager; + +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.common.model.HoodieAvroPayload; + +public class HudiWriteClientProviderProxy implements WriteClientProvider { + + private final HudiClientManager clientManager; + + private final Integer index; + + private final String tableName; + + private final SeaTunnelRowType seaTunnelRowType; + + public HudiWriteClientProviderProxy( + HudiClientManager clientManager, + SeaTunnelRowType seaTunnelRowType, + int index, + String tableName) { + this.clientManager = clientManager; + this.seaTunnelRowType = seaTunnelRowType; + this.index = index; + this.tableName = tableName; + } + + @Override + public HoodieJavaWriteClient getOrCreateClient() { + return clientManager.getClient(this.index, tableName, seaTunnelRowType); + } + + @Override + public void close() { + if (clientManager.containsClient(tableName, index)) { + clientManager.remove(tableName, index).close(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/WriteClientProvider.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/WriteClientProvider.java new file mode 100644 index 00000000000..70726d33f37 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/client/WriteClientProvider.java @@ -0,0 +1,28 @@ +/* + * 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.hudi.sink.client; + +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.common.model.HoodieAvroPayload; + +public interface WriteClientProvider { + + HoodieJavaWriteClient getOrCreateClient(); + + void close(); +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/commiter/HudiSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/commiter/HudiSinkAggregatedCommitter.java new file mode 100644 index 00000000000..beba719c76d --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/commiter/HudiSinkAggregatedCommitter.java @@ -0,0 +1,102 @@ +/* + * 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.hudi.sink.commiter; + +import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.client.HudiWriteClientProvider; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiCommitInfo; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; +import java.util.Stack; +import java.util.stream.Collectors; + +@Slf4j +public class HudiSinkAggregatedCommitter + implements SinkAggregatedCommitter { + + private final HudiTableConfig tableConfig; + + private final HudiWriteClientProvider writeClientProvider; + + public HudiSinkAggregatedCommitter( + HudiTableConfig tableConfig, + HudiSinkConfig sinkConfig, + SeaTunnelRowType seaTunnelRowType) { + this.tableConfig = tableConfig; + this.writeClientProvider = + new HudiWriteClientProvider( + sinkConfig, tableConfig.getTableName(), seaTunnelRowType); + } + + @Override + public List commit( + List aggregatedCommitInfo) throws IOException { + aggregatedCommitInfo = + aggregatedCommitInfo.stream() + .filter( + commit -> + commit.getHudiCommitInfoList().stream() + .anyMatch( + aggregateCommit -> + !aggregateCommit + .getWriteStatusList() + .isEmpty() + && !writeClientProvider + .getOrCreateClient() + .commit( + aggregateCommit + .getWriteInstantTime(), + aggregateCommit + .getWriteStatusList()))) + .collect(Collectors.toList()); + log.debug( + "hudi records have been committed, error commit info are {}", aggregatedCommitInfo); + return aggregatedCommitInfo; + } + + @Override + public HudiAggregatedCommitInfo combine(List commitInfos) { + return new HudiAggregatedCommitInfo(commitInfos); + } + + @Override + public void abort(List aggregatedCommitInfo) throws Exception { + writeClientProvider.getOrCreateClient().rollbackFailedWrites(); + // rollback force commit + for (HudiAggregatedCommitInfo hudiAggregatedCommitInfo : aggregatedCommitInfo) { + for (HudiCommitInfo commitInfo : hudiAggregatedCommitInfo.getHudiCommitInfoList()) { + Stack forceCommitTime = commitInfo.getForceCommitTime(); + while (!forceCommitTime.isEmpty()) { + writeClientProvider.getOrCreateClient().rollback(forceCommitTime.pop()); + } + } + } + } + + @Override + public void close() { + writeClientProvider.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/committer/HudiSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/committer/HudiSinkAggregatedCommitter.java deleted file mode 100644 index 9df2490545b..00000000000 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/committer/HudiSinkAggregatedCommitter.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.hudi.sink.committer; - -import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.client.HoodieJavaWriteClient; -import org.apache.hudi.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.engine.EngineType; -import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.config.HoodieArchivalConfig; -import org.apache.hudi.config.HoodieCleanConfig; -import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; - -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.AvroSchemaConverter.convertToSchema; - -@Slf4j -public class HudiSinkAggregatedCommitter - implements SinkAggregatedCommitter { - - private HoodieJavaWriteClient writeClient; - - private final HoodieWriteConfig cfg; - - private final HadoopStorageConfiguration hudiStorageConfiguration; - - public HudiSinkAggregatedCommitter( - HudiSinkConfig hudiSinkConfig, SeaTunnelRowType seaTunnelRowType) { - - Configuration hadoopConf = new Configuration(); - if (hudiSinkConfig.getConfFilesPath() != null) { - hadoopConf = HudiUtil.getConfiguration(hudiSinkConfig.getConfFilesPath()); - } - hudiStorageConfiguration = new HadoopStorageConfiguration(hadoopConf); - cfg = - HoodieWriteConfig.newBuilder() - .withEmbeddedTimelineServerEnabled(false) - .withEngineType(EngineType.JAVA) - .withPath(hudiSinkConfig.getTableDfsPath()) - .withSchema(convertToSchema(seaTunnelRowType).toString()) - .withParallelism( - hudiSinkConfig.getInsertShuffleParallelism(), - hudiSinkConfig.getUpsertShuffleParallelism()) - .forTable(hudiSinkConfig.getTableName()) - .withIndexConfig( - HoodieIndexConfig.newBuilder() - .withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()) - .withArchivalConfig( - HoodieArchivalConfig.newBuilder() - .archiveCommitsWith( - hudiSinkConfig.getMinCommitsToKeep(), - hudiSinkConfig.getMaxCommitsToKeep()) - .build()) - .withCleanConfig( - HoodieCleanConfig.newBuilder() - .withAutoClean(true) - .withAsyncClean(false) - .build()) - .build(); - } - - @Override - public List commit( - List aggregatedCommitInfo) throws IOException { - writeClient = - new HoodieJavaWriteClient<>( - new HoodieJavaEngineContext(hudiStorageConfiguration), cfg); - aggregatedCommitInfo = - aggregatedCommitInfo.stream() - .filter( - commit -> - commit.getHudiCommitInfoList().stream() - .anyMatch( - aggreeCommit -> - !writeClient.commit( - aggreeCommit - .getInstantTime(), - aggreeCommit - .getWriteStatusList()))) - .collect(Collectors.toList()); - - return aggregatedCommitInfo; - } - - @Override - public HudiAggregatedCommitInfo combine(List commitInfos) { - return new HudiAggregatedCommitInfo(commitInfos); - } - - @Override - public void abort(List aggregatedCommitInfo) throws Exception { - writeClient.rollbackFailedWrites(); - } - - @Override - public void close() { - if (writeClient != null) { - writeClient.close(); - } - } -} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/AvroSchemaConverter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java similarity index 97% rename from seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/AvroSchemaConverter.java rename to seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java index 147456fbbd8..addbf8491f9 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/AvroSchemaConverter.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/AvroSchemaConverter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer; +package org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -29,11 +29,12 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import java.io.Serializable; import java.util.Arrays; import java.util.List; /** Converts an Avro schema into Seatunnel's type information. */ -public class AvroSchemaConverter { +public class AvroSchemaConverter implements Serializable { private AvroSchemaConverter() { // private diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/HudiRecordConverter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/HudiRecordConverter.java new file mode 100644 index 00000000000..d6ed7e81eb1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/HudiRecordConverter.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieKeyException; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.convertToSchema; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.RowDataToAvroConverters.createConverter; + +public class HudiRecordConverter implements Serializable { + + private static final String DEFAULT_PARTITION_PATH = "default"; + + private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; + + private static final String NULL_RECORD_KEY_PLACEHOLDER = "__null__"; + + private static final String EMPTY_RECORD_KEY_PLACEHOLDER = "__empty__"; + + public HoodieRecord convertRow( + Schema schema, + SeaTunnelRowType seaTunnelRowType, + SeaTunnelRow element, + HudiTableConfig hudiTableConfig) { + GenericRecord rec = new GenericData.Record(schema); + for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { + rec.put( + seaTunnelRowType.getFieldNames()[i], + createConverter(seaTunnelRowType.getFieldType(i)) + .convert( + convertToSchema(seaTunnelRowType.getFieldType(i)), + element.getField(i))); + } + return new HoodieAvroRecord<>( + getHoodieKey(element, seaTunnelRowType, hudiTableConfig), + new HoodieAvroPayload(Option.of(rec))); + } + + public HoodieKey getHoodieKey( + SeaTunnelRow element, + SeaTunnelRowType seaTunnelRowType, + HudiTableConfig hudiTableConfig) { + String partitionPath = + hudiTableConfig.getPartitionFields() == null + ? "" + : getRecordPartitionPath(element, seaTunnelRowType, hudiTableConfig); + String rowKey = + hudiTableConfig.getRecordKeyFields() == null + && hudiTableConfig.getOpType().equals(WriteOperationType.INSERT) + ? UUID.randomUUID().toString() + : getRecordKey(element, seaTunnelRowType, hudiTableConfig); + return new HoodieKey(rowKey, partitionPath); + } + + public String getRecordKey( + SeaTunnelRow element, + SeaTunnelRowType seaTunnelRowType, + HudiTableConfig hudiTableConfig) { + boolean keyIsNullEmpty = true; + StringBuilder recordKey = new StringBuilder(); + for (String recordKeyField : hudiTableConfig.getRecordKeyFields().split(",")) { + String recordKeyValue = + getNestedFieldValAsString(element, seaTunnelRowType, recordKeyField); + recordKeyField = recordKeyField.toLowerCase(); + if (recordKeyValue == null) { + recordKey + .append(recordKeyField) + .append(":") + .append(NULL_RECORD_KEY_PLACEHOLDER) + .append(","); + } else if (recordKeyValue.isEmpty()) { + recordKey + .append(recordKeyField) + .append(":") + .append(EMPTY_RECORD_KEY_PLACEHOLDER) + .append(","); + } else { + recordKey.append(recordKeyField).append(":").append(recordKeyValue).append(","); + keyIsNullEmpty = false; + } + } + recordKey.deleteCharAt(recordKey.length() - 1); + if (keyIsNullEmpty) { + throw new HoodieKeyException( + "recordKey values: \"" + + recordKey + + "\" for fields: " + + hudiTableConfig.getRecordKeyFields() + + " cannot be entirely null or empty."); + } + return recordKey.toString(); + } + + public String getRecordPartitionPath( + SeaTunnelRow element, + SeaTunnelRowType seaTunnelRowType, + HudiTableConfig hudiTableConfig) { + if (hudiTableConfig.getPartitionFields().isEmpty()) { + return ""; + } + + StringBuilder partitionPath = new StringBuilder(); + String[] avroPartitionPathFields = hudiTableConfig.getPartitionFields().split(","); + for (String partitionPathField : avroPartitionPathFields) { + String fieldVal = + getNestedFieldValAsString(element, seaTunnelRowType, partitionPathField); + if (fieldVal == null || fieldVal.isEmpty()) { + partitionPath.append(partitionPathField).append("=").append(DEFAULT_PARTITION_PATH); + } else { + partitionPath.append(partitionPathField).append("=").append(fieldVal); + } + partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR); + } + partitionPath.deleteCharAt(partitionPath.length() - 1); + return partitionPath.toString(); + } + + public String getNestedFieldValAsString( + SeaTunnelRow element, SeaTunnelRowType seaTunnelRowType, String fieldName) { + Object value = null; + + if (Arrays.stream(seaTunnelRowType.getFieldNames()) + .collect(Collectors.toList()) + .contains(fieldName)) { + value = element.getField(seaTunnelRowType.indexOf(fieldName)); + } + return StringUtils.objToString(value); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/RowDataToAvroConverters.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java similarity index 98% rename from seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/RowDataToAvroConverters.java rename to seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java index 7cf50deea88..a48179fdb7a 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/RowDataToAvroConverters.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/convert/RowDataToAvroConverters.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer; +package org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -40,10 +40,10 @@ import java.util.List; import java.util.Map; -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.AvroSchemaConverter.extractValueTypeToAvroMap; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.extractValueTypeToAvroMap; /** Tool class used to convert from {@link SeaTunnelRow} to Avro {@link GenericRecord}. */ -public class RowDataToAvroConverters { +public class RowDataToAvroConverters implements Serializable { // -------------------------------------------------------------------------------- // Runtime Converters diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiAggregatedCommitInfo.java new file mode 100644 index 00000000000..348a040be65 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiAggregatedCommitInfo.java @@ -0,0 +1,31 @@ +/* + * 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.hudi.sink.state; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.List; + +@Data +@AllArgsConstructor +public class HudiAggregatedCommitInfo implements Serializable { + + private final List hudiCommitInfoList; +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiCommitInfo.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiCommitInfo.java new file mode 100644 index 00000000000..0357931bb08 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiCommitInfo.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hudi.sink.state; + +import org.apache.hudi.client.WriteStatus; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; +import java.util.List; +import java.util.Stack; + +@Data +@AllArgsConstructor +public class HudiCommitInfo implements Serializable { + + private final String writeInstantTime; + + private final List writeStatusList; + + private final Stack forceCommitTime; +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiSinkState.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiSinkState.java new file mode 100644 index 00000000000..7983dd99e95 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/state/HudiSinkState.java @@ -0,0 +1,32 @@ +/* + * 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.hudi.sink.state; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; + +@Data +@AllArgsConstructor +public class HudiSinkState implements Serializable { + + private long checkpointId; + + private HudiCommitInfo hudiCommitInfo; +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiRecordWriter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiRecordWriter.java new file mode 100644 index 00000000000..7eb3ab546b7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiRecordWriter.java @@ -0,0 +1,271 @@ +/* + * 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.hudi.sink.writer; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiConnectorException; +import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiErrorCode; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.client.WriteClientProvider; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.HudiRecordConverter; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiCommitInfo; + +import org.apache.avro.Schema; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Stack; + +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.convertToSchema; + +@Slf4j +public class HudiRecordWriter implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(HudiRecordWriter.class); + + private final HudiTableConfig hudiTableConfig; + + private final WriteClientProvider clientProvider; + + private final HudiRecordConverter recordConverter; + + private final SeaTunnelRowType seaTunnelRowType; + + private final boolean autoCommit; + + private Schema schema; + + private transient int batchCount = 0; + + private final List> writeRecords; + + private Stack forceCommitTime; + + private String writeInstantTime; + + private List writeStatusList; + + private transient volatile boolean closed = false; + + private transient volatile Exception flushException; + + public HudiRecordWriter( + HudiSinkConfig hudiSinkConfig, + HudiTableConfig hudiTableConfig, + WriteClientProvider clientProvider, + SeaTunnelRowType seaTunnelRowType) { + this.hudiTableConfig = hudiTableConfig; + this.autoCommit = hudiSinkConfig.isAutoCommit(); + this.clientProvider = clientProvider; + this.seaTunnelRowType = seaTunnelRowType; + this.writeRecords = new ArrayList<>(); + this.writeStatusList = new ArrayList<>(); + this.forceCommitTime = new Stack<>(); + this.recordConverter = new HudiRecordConverter(); + } + + public HudiRecordWriter( + HudiSinkConfig sinkConfig, + HudiTableConfig tableConfig, + WriteClientProvider writeClientProvider, + SeaTunnelRowType seaTunnelRowType, + HudiCommitInfo hudiCommitInfo) { + this(sinkConfig, tableConfig, writeClientProvider, seaTunnelRowType); + this.writeInstantTime = hudiCommitInfo.getWriteInstantTime(); + this.writeStatusList = hudiCommitInfo.getWriteStatusList(); + } + + public void open() { + this.schema = new Schema.Parser().parse(convertToSchema(seaTunnelRowType).toString()); + try { + HoodieJavaWriteClient writeClient = + clientProvider.getOrCreateClient(); + if (StringUtils.nonEmpty(writeInstantTime) && Objects.nonNull(writeStatusList)) { + if (!writeClient.commit(writeInstantTime, writeStatusList)) { + LOG.warn("Failed to commit history data."); + } + } + } catch (Exception e) { + throw new HudiConnectorException( + CommonErrorCodeDeprecated.WRITER_OPERATION_FAILED, + "Commit history data error.", + e); + } + } + + public void writeRecord(SeaTunnelRow record) { + checkFlushException(); + try { + prepareRecords(record); + batchCount++; + if (hudiTableConfig.getBatchSize() > 0 + && batchCount >= hudiTableConfig.getBatchSize()) { + flush(true); + } + } catch (Exception e) { + throw new HudiConnectorException( + CommonErrorCodeDeprecated.WRITER_OPERATION_FAILED, + "Writing records to Hudi failed.", + e); + } + } + + public synchronized void flush(boolean isNeedForceCommit) { + if (batchCount == 0) { + log.debug("No data needs to be refreshed, waiting for incoming data."); + return; + } + checkFlushException(); + HoodieJavaWriteClient writeClient = clientProvider.getOrCreateClient(); + if (autoCommit || writeInstantTime == null) { + writeInstantTime = writeClient.startCommit(); + } + List currentWriteStatusList; + // write records + switch (hudiTableConfig.getOpType()) { + case INSERT: + currentWriteStatusList = writeClient.insert(writeRecords, writeInstantTime); + break; + case UPSERT: + currentWriteStatusList = writeClient.upsert(writeRecords, writeInstantTime); + break; + case BULK_INSERT: + currentWriteStatusList = writeClient.bulkInsert(writeRecords, writeInstantTime); + break; + default: + throw new HudiConnectorException( + HudiErrorCode.UNSUPPORTED_OPERATION, + "Unsupported operation type: " + hudiTableConfig.getOpType()); + } + if (!autoCommit) { + this.writeStatusList.addAll(currentWriteStatusList); + } + /** + * when the batch size of temporary records is reached, commit is forced here, even if + * configured not to be auto commit. because a timeline supports only one commit. + */ + forceCommit(isNeedForceCommit, autoCommit); + writeRecords.clear(); + batchCount = 0; + } + + public Optional prepareCommit() { + flush(false); + if (!autoCommit) { + return Optional.of( + new HudiCommitInfo(writeInstantTime, writeStatusList, forceCommitTime)); + } + return Optional.empty(); + } + + private void commit() { + if (StringUtils.nonEmpty(writeInstantTime) && !writeStatusList.isEmpty()) { + log.debug( + "Commit hudi records, the instant time is {} and write status are {}", + writeInstantTime, + writeStatusList); + clientProvider.getOrCreateClient().commit(writeInstantTime, writeStatusList); + resetUpsertCommitInfo(); + } + } + + private void forceCommit(boolean isNeedForceCommit, boolean isAutoCommit) { + if (isNeedForceCommit && !isAutoCommit) { + clientProvider.getOrCreateClient().commit(writeInstantTime, writeStatusList); + forceCommitTime.add(writeInstantTime); + resetUpsertCommitInfo(); + } + } + + public HudiCommitInfo snapshotState() { + HudiCommitInfo hudiCommitInfo = + new HudiCommitInfo(writeInstantTime, writeStatusList, forceCommitTime); + // reset commit info in here, because the commit info will be committed in committer. + resetUpsertCommitInfo(); + // reset the force commit stack. + forceCommitTime = new Stack<>(); + return hudiCommitInfo; + } + + protected void resetUpsertCommitInfo() { + writeInstantTime = null; + writeStatusList = new ArrayList<>(); + } + + protected void prepareRecords(SeaTunnelRow element) { + HoodieRecord hoodieAvroPayloadHoodieRecord = + recordConverter.convertRow(schema, seaTunnelRowType, element, hudiTableConfig); + writeRecords.add(hoodieAvroPayloadHoodieRecord); + } + + protected void checkFlushException() { + if (flushException != null) { + throw new HudiConnectorException( + HudiErrorCode.FLUSH_DATA_FAILED, + "Flush records to Hudi failed.", + flushException); + } + } + + /** Executes prepared statement and closes all resources of this instance. */ + public synchronized void close() { + if (!closed) { + closed = true; + try { + flush(false); + if (!autoCommit) { + commit(); + } + } catch (Exception e) { + LOG.warn("Flush records to Hudi failed.", e); + flushException = + new HudiConnectorException( + CommonErrorCodeDeprecated.FLUSH_DATA_FAILED, + "Flush records to Hudi failed.", + e); + } + + try { + if (clientProvider != null) { + clientProvider.close(); + } + } catch (Exception e) { + LOG.warn("Close Hudi record writer failed.", e); + } + } + checkFlushException(); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiSinkWriter.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiSinkWriter.java index 50effd6b446..317215861a2 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/writer/HudiSinkWriter.java @@ -17,168 +17,88 @@ package org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer; +import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; -import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiConnectorException; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hudi.state.HudiSinkState; -import org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.client.HoodieJavaWriteClient; -import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieJavaEngineContext; -import org.apache.hudi.common.engine.EngineType; -import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.common.model.HoodieAvroRecord; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.config.HoodieArchivalConfig; -import org.apache.hudi.config.HoodieCleanConfig; -import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieKeyException; -import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.HudiClientManager; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.HudiMultiTableResourceManager; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.client.HudiWriteClientProvider; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.client.HudiWriteClientProviderProxy; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.client.WriteClientProvider; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hudi.sink.state.HudiSinkState; import lombok.extern.slf4j.Slf4j; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.UUID; -import java.util.stream.Collectors; - -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.AvroSchemaConverter.convertToSchema; -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.RowDataToAvroConverters.createConverter; @Slf4j public class HudiSinkWriter implements SinkWriter, - SupportMultiTableSinkWriter { + SupportMultiTableSinkWriter { - public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; - protected static final String DEFAULT_PARTITION_PATH = "default"; - protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__"; - protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__"; - private final HoodieJavaWriteClient writeClient; - private final WriteOperationType opType; - private final Schema schema; - private final SeaTunnelRowType seaTunnelRowType; - private final HudiSinkConfig hudiSinkConfig; - private final List> hoodieRecords; - private transient List writeStatusList; - private transient String instantTime; - private transient int batchCount = 0; - private transient volatile boolean closed = false; - private transient volatile Exception flushException; + private WriteClientProvider writeClientProvider; - public HudiSinkWriter( - SinkWriter.Context context, - SeaTunnelRowType seaTunnelRowType, - HudiSinkConfig hudiSinkConfig, - List hudiSinkState) - throws IOException { + private final HudiSinkConfig sinkConfig; - this.hoodieRecords = new ArrayList<>(30); - this.seaTunnelRowType = seaTunnelRowType; - this.schema = new Schema.Parser().parse(convertToSchema(seaTunnelRowType).toString()); - this.opType = hudiSinkConfig.getOpType(); - this.hudiSinkConfig = hudiSinkConfig; - Configuration hadoopConf = new Configuration(); - if (hudiSinkConfig.getConfFilesPath() != null) { - hadoopConf = HudiUtil.getConfiguration(hudiSinkConfig.getConfFilesPath()); - } - HadoopStorageConfiguration hudiStorageConfiguration = - new HadoopStorageConfiguration(hadoopConf); + private final HudiTableConfig tableConfig; + + private final SeaTunnelRowType seaTunnelRowType; - // initialize the table, if not done already - Path path = new Path(hudiSinkConfig.getTableDfsPath()); - FileSystem fs = - HadoopFSUtils.getFs(hudiSinkConfig.getTableDfsPath(), hudiStorageConfiguration); - HoodieTableMetaClient.withPropertyBuilder() - .setTableType(hudiSinkConfig.getTableType()) - .setTableName(hudiSinkConfig.getTableName()) - .setPayloadClassName(HoodieAvroPayload.class.getName()) - .initTable(hudiStorageConfiguration, hudiSinkConfig.getTableDfsPath()); - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder() - .withEmbeddedTimelineServerEnabled(false) - .withEngineType(EngineType.JAVA) - .withPath(hudiSinkConfig.getTableDfsPath()) - .withSchema(convertToSchema(seaTunnelRowType).toString()) - .withParallelism( - hudiSinkConfig.getInsertShuffleParallelism(), - hudiSinkConfig.getUpsertShuffleParallelism()) - .forTable(hudiSinkConfig.getTableName()) - .withIndexConfig( - HoodieIndexConfig.newBuilder() - .withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()) - .withArchivalConfig( - HoodieArchivalConfig.newBuilder() - .archiveCommitsWith( - hudiSinkConfig.getMinCommitsToKeep(), - hudiSinkConfig.getMaxCommitsToKeep()) - .build()) - .withAutoCommit(false) - .withCleanConfig( - HoodieCleanConfig.newBuilder() - .withAutoClean(true) - .withAsyncClean(false) - .build()) - .build(); + private HudiRecordWriter hudiRecordWriter; - writeClient = - new HoodieJavaWriteClient<>( - new HoodieJavaEngineContext(hudiStorageConfiguration), cfg); + private transient boolean isOpen; + public HudiSinkWriter( + Context context, + SeaTunnelRowType seaTunnelRowType, + HudiSinkConfig sinkConfig, + HudiTableConfig tableConfig, + List hudiSinkState) { + this.sinkConfig = sinkConfig; + this.tableConfig = tableConfig; + this.seaTunnelRowType = seaTunnelRowType; + this.writeClientProvider = + new HudiWriteClientProvider( + sinkConfig, tableConfig.getTableName(), seaTunnelRowType); if (!hudiSinkState.isEmpty()) { - writeClient.commit( - hudiSinkState.get(0).getHudiCommitInfo().getInstantTime(), - hudiSinkState.get(0).getHudiCommitInfo().getWriteStatusList()); + this.hudiRecordWriter = + new HudiRecordWriter( + sinkConfig, + tableConfig, + writeClientProvider, + seaTunnelRowType, + hudiSinkState.get(0).getHudiCommitInfo()); + } else { + this.hudiRecordWriter = + new HudiRecordWriter( + sinkConfig, tableConfig, writeClientProvider, seaTunnelRowType); } } @Override public void write(SeaTunnelRow element) throws IOException { - checkFlushException(); - - batchCount++; - prepareRecords(element); - - if (batchCount >= hudiSinkConfig.getMaxCommitsToKeep()) { - flush(); - } + tryOpen(); + hudiRecordWriter.writeRecord(element); } @Override - public Optional prepareCommit() { - flush(); - return Optional.of(new HudiCommitInfo(instantTime, writeStatusList)); + public List snapshotState(long checkpointId) throws IOException { + return Collections.singletonList( + new HudiSinkState(checkpointId, hudiRecordWriter.snapshotState())); } @Override - public List snapshotState(long checkpointId) throws IOException { - return Collections.singletonList( - new HudiSinkState(checkpointId, new HudiCommitInfo(instantTime, writeStatusList))); + public Optional prepareCommit() throws IOException { + tryOpen(); + return hudiRecordWriter.prepareCommit(); } @Override @@ -186,155 +106,36 @@ public void abortPrepare() {} @Override public void close() throws IOException { - if (!closed) { - - if (batchCount > 0) { - try { - flush(); - } catch (Exception e) { - log.warn("Writing records to Hudi failed.", e); - throw new HudiConnectorException( - CommonErrorCode.WRITE_SEATUNNEL_ROW_ERROR, - "Writing records to hudi failed.", - e); - } - } - if (writeClient != null) { - writeClient.close(); - } - closed = true; - checkFlushException(); - } + hudiRecordWriter.close(); } - private void prepareRecords(SeaTunnelRow element) { - - hoodieRecords.add(convertRow(element)); - } - - private HoodieRecord convertRow(SeaTunnelRow element) { - GenericRecord rec = new GenericData.Record(schema); - for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { - rec.put( - seaTunnelRowType.getFieldNames()[i], - createConverter(seaTunnelRowType.getFieldType(i)) - .convert( - convertToSchema(seaTunnelRowType.getFieldType(i)), - element.getField(i))); - } - return new HoodieAvroRecord<>( - getHoodieKey(element, seaTunnelRowType), new HoodieAvroPayload(Option.of(rec))); - } - - private HoodieKey getHoodieKey(SeaTunnelRow element, SeaTunnelRowType seaTunnelRowType) { - String partitionPath = - hudiSinkConfig.getPartitionFields() == null - ? "" - : getRecordPartitionPath(element, seaTunnelRowType); - String rowKey = - hudiSinkConfig.getRecordKeyFields() == null - && hudiSinkConfig.getOpType().equals(WriteOperationType.INSERT) - ? UUID.randomUUID().toString() - : getRecordKey(element, seaTunnelRowType); - return new HoodieKey(rowKey, partitionPath); - } - - private String getRecordKey(SeaTunnelRow element, SeaTunnelRowType seaTunnelRowType) { - boolean keyIsNullEmpty = true; - StringBuilder recordKey = new StringBuilder(); - for (String recordKeyField : hudiSinkConfig.getRecordKeyFields().split(",")) { - String recordKeyValue = - getNestedFieldValAsString(element, seaTunnelRowType, recordKeyField); - recordKeyField = recordKeyField.toLowerCase(); - if (recordKeyValue == null) { - recordKey - .append(recordKeyField) - .append(":") - .append(NULL_RECORDKEY_PLACEHOLDER) - .append(","); - } else if (recordKeyValue.isEmpty()) { - recordKey - .append(recordKeyField) - .append(":") - .append(EMPTY_RECORDKEY_PLACEHOLDER) - .append(","); - } else { - recordKey.append(recordKeyField).append(":").append(recordKeyValue).append(","); - keyIsNullEmpty = false; - } - } - recordKey.deleteCharAt(recordKey.length() - 1); - if (keyIsNullEmpty) { - throw new HoodieKeyException( - "recordKey values: \"" - + recordKey - + "\" for fields: " - + hudiSinkConfig.getRecordKeyFields() - + " cannot be entirely null or empty."); - } - return recordKey.toString(); - } - - private String getRecordPartitionPath(SeaTunnelRow element, SeaTunnelRowType seaTunnelRowType) { - if (hudiSinkConfig.getPartitionFields().isEmpty()) { - return ""; - } - - StringBuilder partitionPath = new StringBuilder(); - String[] avroPartitionPathFields = hudiSinkConfig.getPartitionFields().split(","); - for (String partitionPathField : avroPartitionPathFields) { - String fieldVal = - getNestedFieldValAsString(element, seaTunnelRowType, partitionPathField); - if (fieldVal == null || fieldVal.isEmpty()) { - partitionPath.append(partitionPathField).append("=").append(DEFAULT_PARTITION_PATH); - } else { - partitionPath.append(partitionPathField).append("=").append(fieldVal); - } - partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR); - } - partitionPath.deleteCharAt(partitionPath.length() - 1); - return partitionPath.toString(); - } - - private String getNestedFieldValAsString( - SeaTunnelRow element, SeaTunnelRowType seaTunnelRowType, String fieldName) { - Object value = null; - - if (Arrays.stream(seaTunnelRowType.getFieldNames()) - .collect(Collectors.toList()) - .contains(fieldName)) { - value = element.getField(seaTunnelRowType.indexOf(fieldName)); - } - return StringUtils.objToString(value); + @Override + public MultiTableResourceManager initMultiTableResourceManager( + int tableSize, int queueSize) { + return new HudiMultiTableResourceManager(new HudiClientManager(sinkConfig)); } - public synchronized void flush() { - checkFlushException(); - instantTime = writeClient.startCommit(); - switch (opType) { - case INSERT: - writeStatusList = writeClient.insert(hoodieRecords, instantTime); - break; - case UPSERT: - writeStatusList = writeClient.upsert(hoodieRecords, instantTime); - break; - case BULK_INSERT: - writeStatusList = writeClient.bulkInsert(hoodieRecords, instantTime); - break; - default: - throw new HudiConnectorException( - CommonErrorCode.OPERATION_NOT_SUPPORTED, - "Unsupported operation type: " + opType); - } - batchCount = 0; + @Override + public void setMultiTableResourceManager( + MultiTableResourceManager multiTableResourceManager, + int queueIndex) { + log.info("multi table resource manager is {}", multiTableResourceManager); + this.hudiRecordWriter.close(); + this.writeClientProvider = + new HudiWriteClientProviderProxy( + multiTableResourceManager.getSharedResource().get(), + seaTunnelRowType, + queueIndex, + tableConfig.getTableName()); + this.hudiRecordWriter = + new HudiRecordWriter( + sinkConfig, tableConfig, writeClientProvider, seaTunnelRowType); } - private void checkFlushException() { - if (flushException != null) { - throw new HudiConnectorException( - CommonErrorCode.WRITE_SEATUNNEL_ROW_ERROR, - "Writing records to Hudi failed.", - flushException); + private void tryOpen() { + if (!isOpen) { + isOpen = true; + hudiRecordWriter.open(); } } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiCatalogUtil.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiCatalogUtil.java new file mode 100644 index 00000000000..659118e529f --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiCatalogUtil.java @@ -0,0 +1,37 @@ +/* + * 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.hudi.util; + +import org.apache.seatunnel.api.table.catalog.TablePath; + +import org.apache.commons.lang3.StringUtils; + +public class HudiCatalogUtil { + + public static String inferTablePath( + String tableDfsPath, String databaseName, String tableName) { + if (StringUtils.isEmpty(databaseName)) { + return String.format("%s/%s", tableDfsPath, tableName); + } + return String.format("%s/%s/%s", tableDfsPath, databaseName, tableName); + } + + public static String inferTablePath(String tableDfsPath, TablePath tablePath) { + return inferTablePath(tableDfsPath, tablePath.getDatabaseName(), tablePath.getTableName()); + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java index 740a4e78083..fe6cbe3e206 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiTableConfig; import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiConnectorException; import org.apache.hadoop.conf.Configuration; @@ -29,21 +31,41 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import java.io.IOException; import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Optional; import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiErrorCode.TABLE_CONFIG_NOT_FOUND; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.convertToSchema; +import static org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiCatalogUtil.inferTablePath; public class HudiUtil { public static Configuration getConfiguration(String confPaths) { Configuration configuration = new Configuration(); - Arrays.stream(confPaths.split(";")) - .forEach(file -> configuration.addResource(new Path(file))); + if (confPaths != null) { + Arrays.stream(confPaths.split(";")) + .forEach(file -> configuration.addResource(new Path(file))); + } return configuration; } @@ -114,4 +136,71 @@ public static void initKerberosAuthentication( e); } } + + public static HoodieJavaWriteClient createHoodieJavaWriteClient( + HudiSinkConfig hudiSinkConfig, SeaTunnelRowType seaTunnelRowType, String tableName) { + List tableList = hudiSinkConfig.getTableList(); + Optional hudiTableConfig = + tableList.stream() + .filter(table -> table.getTableName().equals(tableName)) + .findFirst(); + if (!hudiTableConfig.isPresent()) { + throw new HudiConnectorException( + TABLE_CONFIG_NOT_FOUND, + "The corresponding table " + + tableName + + " is not found in the table list of hudi sink config."); + } + Configuration hadoopConf = getConfiguration(hudiSinkConfig.getConfFilesPath()); + + HudiTableConfig hudiTable = hudiTableConfig.get(); + HoodieWriteConfig.Builder writeConfigBuilder = HoodieWriteConfig.newBuilder(); + // build index config + if (Objects.nonNull(hudiTable.getIndexClassName())) { + writeConfigBuilder.withIndexConfig( + HoodieIndexConfig.newBuilder() + .withIndexClass(hudiTable.getIndexClassName()) + .build()); + } else { + writeConfigBuilder.withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(hudiTable.getIndexType()).build()); + } + HoodieWriteConfig cfg = + writeConfigBuilder + .withEngineType(EngineType.JAVA) + .withPath( + inferTablePath( + hudiSinkConfig.getTableDfsPath(), + hudiTable.getDatabase(), + hudiTable.getTableName())) + .withSchema(convertToSchema(seaTunnelRowType).toString()) + .withParallelism( + hudiTable.getInsertShuffleParallelism(), + hudiTable.getUpsertShuffleParallelism()) + .forTable(hudiTable.getTableName()) + .withArchivalConfig( + HoodieArchivalConfig.newBuilder() + .archiveCommitsWith( + hudiTable.getMinCommitsToKeep(), + hudiTable.getMaxCommitsToKeep()) + .build()) + .withAutoCommit(hudiSinkConfig.isAutoCommit()) + .withCleanConfig( + HoodieCleanConfig.newBuilder() + .withAutoClean(true) + .withAsyncClean(false) + .build()) + .withEmbeddedTimelineServerEnabled(false) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder() + .approxRecordSize(hudiTable.getRecordByteSize()) + .build()) + .withStorageConfig( + HoodieStorageConfig.newBuilder() + .parquetCompressionCodec(CompressionCodecName.SNAPPY.name()) + .build()) + .build(); + return new HoodieJavaWriteClient<>( + new HoodieJavaEngineContext(new HadoopStorageConfiguration(hadoopConf)), cfg); + } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/SchemaUtil.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/SchemaUtil.java new file mode 100644 index 00000000000..cd8eb620e83 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/SchemaUtil.java @@ -0,0 +1,147 @@ +/* + * 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.hudi.util; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonError; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.List; + +public class SchemaUtil { + + public static SeaTunnelDataType convertSeaTunnelType(String field, Schema schema) { + switch (schema.getType()) { + case RECORD: + return convertStructType(schema); + case ENUM: + case STRING: + case NULL: + return BasicType.STRING_TYPE; + case ARRAY: + return convertListType(field, schema.getElementType()); + case MAP: + return convertMapType(field, schema); + case BYTES: + case FIXED: + // logical decimal type + if (schema.getLogicalType() instanceof LogicalTypes.Decimal) { + final LogicalTypes.Decimal decimalType = + (LogicalTypes.Decimal) schema.getLogicalType(); + return new DecimalType(decimalType.getPrecision(), decimalType.getScale()); + } + return PrimitiveByteArrayType.INSTANCE; + case INT: + // logical date and time type + final org.apache.avro.LogicalType logicalType = schema.getLogicalType(); + if (logicalType == LogicalTypes.date()) { + return LocalTimeType.LOCAL_DATE_TYPE; + } else if (logicalType == LogicalTypes.timeMillis()) { + return LocalTimeType.LOCAL_TIME_TYPE; + } + return BasicType.INT_TYPE; + case LONG: + // logical timestamp type + if (schema.getLogicalType() == LogicalTypes.timestampMillis()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } else if (schema.getLogicalType() == LogicalTypes.localTimestampMillis()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } else if (schema.getLogicalType() == LogicalTypes.timestampMicros()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } else if (schema.getLogicalType() == LogicalTypes.localTimestampMicros()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } else if (schema.getLogicalType() == LogicalTypes.timeMillis()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } else if (schema.getLogicalType() == LogicalTypes.timeMicros()) { + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + } + return BasicType.LONG_TYPE; + case FLOAT: + return BasicType.FLOAT_TYPE; + case DOUBLE: + return BasicType.DOUBLE_TYPE; + case BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case UNION: + final Schema actualSchema; + if (schema.getTypes().size() == 2 + && schema.getTypes().get(0).getType() == Schema.Type.NULL) { + actualSchema = schema.getTypes().get(1); + } else if (schema.getTypes().size() == 2 + && schema.getTypes().get(1).getType() == Schema.Type.NULL) { + actualSchema = schema.getTypes().get(0); + } else if (schema.getTypes().size() == 1) { + actualSchema = schema.getTypes().get(0); + } else { + throw CommonError.convertToSeaTunnelTypeError( + "Hudi", schema.getType().name(), field); + } + return convertSeaTunnelType(field, actualSchema); + default: + throw CommonError.convertToSeaTunnelTypeError( + "Hudi", schema.getType().name(), field); + } + } + + private static MapType convertMapType(String field, Schema schema) { + return new MapType( + convertSeaTunnelType(field, schema.getElementType()), + convertSeaTunnelType(field, schema.getValueType())); + } + + private static SeaTunnelRowType convertStructType(Schema schema) { + List fields = schema.getFields(); + List fieldNames = new ArrayList<>(fields.size()); + List> fieldTypes = new ArrayList<>(fields.size()); + for (Schema.Field field : fields) { + fieldNames.add(field.name()); + fieldTypes.add(convertSeaTunnelType(field.name(), field.schema())); + } + return new SeaTunnelRowType( + fieldNames.toArray(new String[0]), fieldTypes.toArray(new SeaTunnelDataType[0])); + } + + private static ArrayType convertListType(String field, Schema schema) { + switch (schema.getElementType().getType()) { + case BOOLEAN: + return ArrayType.BOOLEAN_ARRAY_TYPE; + case INT: + return ArrayType.INT_ARRAY_TYPE; + case LONG: + return ArrayType.LONG_ARRAY_TYPE; + case FLOAT: + return ArrayType.FLOAT_ARRAY_TYPE; + case DOUBLE: + return ArrayType.DOUBLE_ARRAY_TYPE; + case STRING: + return ArrayType.STRING_ARRAY_TYPE; + default: + throw CommonError.convertToSeaTunnelTypeError("Hudi", schema.toString(), field); + } + } +} diff --git a/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/HudiTest.java b/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/HudiTest.java index 1da165bdcbb..82e85fcf4e2 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/HudiTest.java +++ b/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/HudiTest.java @@ -67,8 +67,8 @@ import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.AvroSchemaConverter.convertToSchema; -import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.writer.RowDataToAvroConverters.createConverter; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.AvroSchemaConverter.convertToSchema; +import static org.apache.seatunnel.connectors.seatunnel.hudi.sink.convert.RowDataToAvroConverters.createConverter; public class HudiTest { diff --git a/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogTest.java b/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogTest.java new file mode 100644 index 00000000000..7be81e89ba8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hudi/src/test/java/org/apache/seatunnel/connectors/seatunnel/hudi/catalog/HudiCatalogTest.java @@ -0,0 +1,175 @@ +/* + * 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.hudi.catalog; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; + +import org.apache.hadoop.conf.Configuration; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; + +import java.util.Collections; +import java.util.HashMap; + +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.apache.seatunnel.api.table.type.LocalTimeType.LOCAL_DATE_TIME_TYPE; +import static org.apache.seatunnel.api.table.type.LocalTimeType.LOCAL_DATE_TYPE; + +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@Disabled +class HudiCatalogTest { + private static final String CATALOG_NAME = "seatunnel"; + private static final String CATALOG_DIR = "/tmp/seatunnel/hudi"; + + private static HudiCatalog hudicatalog; + + private static final String DATABASE = "st"; + private static final String DEFAULT_DATABASE = "default"; + private static final String TABLE_NAME = "hudi_test"; + + private final TablePath tablePath = TablePath.of(DATABASE, null, TABLE_NAME); + private final TableIdentifier tableIdentifier = + TableIdentifier.of(CATALOG_NAME, DATABASE, null, TABLE_NAME); + + @BeforeAll + static void setUpBeforeClass() throws Exception { + hudicatalog = new HudiCatalog(CATALOG_NAME, new Configuration(), CATALOG_DIR); + hudicatalog.open(); + } + + @AfterAll + static void tearDownAfterClass() throws Exception { + hudicatalog.close(); + } + + @Test + @Order(1) + void getDefaultDatabase() { + Assertions.assertEquals(hudicatalog.getDefaultDatabase(), DEFAULT_DATABASE); + Assertions.assertTrue(hudicatalog.databaseExists(DEFAULT_DATABASE)); + } + + @Test + @Order(2) + void createTable() { + CatalogTable catalogTable = buildAllTypesTable(tableIdentifier); + hudicatalog.createTable(tablePath, catalogTable, true); + Assertions.assertTrue(hudicatalog.tableExists(tablePath)); + } + + @Test + @Order(3) + void databaseExists() { + Assertions.assertTrue(hudicatalog.databaseExists(DATABASE)); + Assertions.assertFalse(hudicatalog.databaseExists("st_not_exists")); + } + + @Test + @Order(4) + void listDatabases() { + hudicatalog.listDatabases().forEach(System.out::println); + Assertions.assertTrue(hudicatalog.listDatabases().contains(DATABASE)); + Assertions.assertTrue(hudicatalog.listDatabases().contains(DEFAULT_DATABASE)); + } + + @Test + @Order(5) + void listTables() { + Assertions.assertTrue(hudicatalog.listTables(DATABASE).contains(TABLE_NAME)); + } + + @Test + @Order(6) + void tableExists() { + Assertions.assertTrue(hudicatalog.tableExists(tablePath)); + Assertions.assertFalse(hudicatalog.tableExists(TablePath.of(DATABASE, "ssssss"))); + } + + @Test + @Order(7) + void getTable() { + CatalogTable table = hudicatalog.getTable(tablePath); + CatalogTable templateTable = buildAllTypesTable(tableIdentifier); + Assertions.assertEquals(table.toString(), templateTable.toString()); + } + + @Test + @Order(8) + void dropTable() { + hudicatalog.dropTable(tablePath, false); + Assertions.assertFalse(hudicatalog.tableExists(tablePath)); + } + + CatalogTable buildAllTypesTable(TableIdentifier tableIdentifier) { + TableSchema.Builder builder = TableSchema.builder(); + builder.column(PhysicalColumn.of("id", BasicType.INT_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "boolean_col", BasicType.BOOLEAN_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "integer_col", BasicType.INT_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of("long_col", BasicType.LONG_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "float_col", BasicType.FLOAT_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "double_col", BasicType.DOUBLE_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of("date_col", LOCAL_DATE_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "timestamp_col", LOCAL_DATE_TIME_TYPE, (Long) null, true, null, null)); + builder.column(PhysicalColumn.of("string_col", STRING_TYPE, (Long) null, true, null, null)); + builder.column( + PhysicalColumn.of( + "binary_col", + PrimitiveByteArrayType.INSTANCE, + (Long) null, + true, + null, + null)); + builder.column( + PhysicalColumn.of( + "decimal_col", new DecimalType(38, 18), (Long) null, true, null, null)); + builder.column(PhysicalColumn.of("dt_col", STRING_TYPE, (Long) null, true, null, null)); + + TableSchema schema = builder.build(); + HashMap options = new HashMap<>(); + options.put("record_key_fields", "id,boolean_col"); + options.put("table_type", "MERGE_ON_READ"); + return CatalogTable.of( + tableIdentifier, schema, options, Collections.singletonList("dt_col"), "null"); + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index f7ec6bd3090..999f5e7fafc 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; @@ -45,6 +46,7 @@ import java.net.URL; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -125,18 +127,7 @@ public List execute(List upstreamDataS sink = ((TableSinkFactory) factory.get()).createSink(context).createSink(); sink.setJobContext(jobContext); } - if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { - SupportSaveMode saveModeSink = (SupportSaveMode) sink; - Optional saveModeHandler = saveModeSink.getSaveModeHandler(); - if (saveModeHandler.isPresent()) { - try (SaveModeHandler handler = saveModeHandler.get()) { - handler.open(); - new SaveModeExecuteWrapper(handler).execute(); - } catch (Exception e) { - throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); - } - } - } + handleSaveMode(sink); DataStreamSink dataStreamSink = stream.getDataStream() .sinkTo(new FlinkSink<>(sink, stream.getCatalogTables().get(0))) @@ -171,4 +162,24 @@ public SeaTunnelSink fallbackCreateSink( source.prepare(pluginConfig); return source; } + + public void handleSaveMode(SeaTunnelSink sink) { + if (sink instanceof SupportSaveMode) { + Optional saveModeHandler = + ((SupportSaveMode) sink).getSaveModeHandler(); + if (saveModeHandler.isPresent()) { + try (SaveModeHandler handler = saveModeHandler.get()) { + handler.open(); + new SaveModeExecuteWrapper(handler).execute(); + } catch (Exception e) { + throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); + } + } + } else if (sink instanceof MultiTableSink) { + Map sinks = ((MultiTableSink) sink).getSinks(); + for (SeaTunnelSink seaTunnelSink : sinks.values()) { + handleSaveMode(seaTunnelSink); + } + } + } } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index b4f192bc618..65e6d3b9f2f 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; @@ -46,6 +47,7 @@ import java.net.URL; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -126,18 +128,7 @@ public List execute(List upstreamDataS sink = ((TableSinkFactory) factory.get()).createSink(context).createSink(); sink.setJobContext(jobContext); } - if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { - SupportSaveMode saveModeSink = (SupportSaveMode) sink; - Optional saveModeHandler = saveModeSink.getSaveModeHandler(); - if (saveModeHandler.isPresent()) { - try (SaveModeHandler handler = saveModeHandler.get()) { - handler.open(); - new SaveModeExecuteWrapper(handler).execute(); - } catch (Exception e) { - throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); - } - } - } + handleSaveMode(sink); DataStreamSink dataStreamSink = stream.getDataStream() .sinkTo( @@ -175,4 +166,24 @@ public SeaTunnelSink fallbackCreateSink( source.prepare(pluginConfig); return source; } + + public void handleSaveMode(SeaTunnelSink sink) { + if (sink instanceof SupportSaveMode) { + Optional saveModeHandler = + ((SupportSaveMode) sink).getSaveModeHandler(); + if (saveModeHandler.isPresent()) { + try (SaveModeHandler handler = saveModeHandler.get()) { + handler.open(); + new SaveModeExecuteWrapper(handler).execute(); + } catch (Exception e) { + throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); + } + } + } else if (sink instanceof MultiTableSink) { + Map sinks = ((MultiTableSink) sink).getSinks(); + for (SeaTunnelSink seaTunnelSink : sinks.values()) { + handleSaveMode(seaTunnelSink); + } + } + } } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 20ca02b6166..48f8cab8e1d 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -44,6 +45,7 @@ import java.net.URL; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -117,18 +119,7 @@ public List execute(List upstreamDataStreams datasetTableInfo.getCatalogTables(), classLoader); // TODO modify checkpoint location - if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { - SupportSaveMode saveModeSink = (SupportSaveMode) sink; - Optional saveModeHandler = saveModeSink.getSaveModeHandler(); - if (saveModeHandler.isPresent()) { - try (SaveModeHandler handler = saveModeHandler.get()) { - handler.open(); - new SaveModeExecuteWrapper(handler).execute(); - } catch (Exception e) { - throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); - } - } - } + handleSaveMode(sink); String applicationId = sparkRuntimeEnvironment.getSparkSession().sparkContext().applicationId(); CatalogTable[] catalogTables = @@ -162,4 +153,24 @@ public SeaTunnelSink fallbackCreateSink( source.prepare(pluginConfig); return source; } + + public void handleSaveMode(SeaTunnelSink sink) { + if (sink instanceof SupportSaveMode) { + Optional saveModeHandler = + ((SupportSaveMode) sink).getSaveModeHandler(); + if (saveModeHandler.isPresent()) { + try (SaveModeHandler handler = saveModeHandler.get()) { + handler.open(); + new SaveModeExecuteWrapper(handler).execute(); + } catch (Exception e) { + throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); + } + } + } else if (sink instanceof MultiTableSink) { + Map sinks = ((MultiTableSink) sink).getSinks(); + for (SeaTunnelSink seaTunnelSink : sinks.values()) { + handleSaveMode(seaTunnelSink); + } + } + } } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 0cca2f8d681..0b54e2a115d 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -45,6 +46,7 @@ import java.net.URL; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -117,18 +119,7 @@ public List execute(List upstreamDataStreams datasetTableInfo.getCatalogTables(), classLoader); // TODO modify checkpoint location - if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { - SupportSaveMode saveModeSink = (SupportSaveMode) sink; - Optional saveModeHandler = saveModeSink.getSaveModeHandler(); - if (saveModeHandler.isPresent()) { - try (SaveModeHandler handler = saveModeHandler.get()) { - handler.open(); - new SaveModeExecuteWrapper(handler).execute(); - } catch (Exception e) { - throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); - } - } - } + handleSaveMode(sink); String applicationId = sparkRuntimeEnvironment.getStreamingContext().sparkContext().applicationId(); CatalogTable[] catalogTables = @@ -163,4 +154,24 @@ public SeaTunnelSink fallbackCreateSink( source.prepare(pluginConfig); return source; } + + public void handleSaveMode(SeaTunnelSink sink) { + if (sink instanceof SupportSaveMode) { + Optional saveModeHandler = + ((SupportSaveMode) sink).getSaveModeHandler(); + if (saveModeHandler.isPresent()) { + try (SaveModeHandler handler = saveModeHandler.get()) { + handler.open(); + new SaveModeExecuteWrapper(handler).execute(); + } catch (Exception e) { + throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); + } + } + } else if (sink instanceof MultiTableSink) { + Map sinks = ((MultiTableSink) sink).getSinks(); + for (SeaTunnelSink seaTunnelSink : sinks.values()) { + handleSaveMode(seaTunnelSink); + } + } + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/pom.xml index bbe1e2187ec..583b16a162d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/pom.xml @@ -25,7 +25,35 @@ connector-hudi-e2e SeaTunnel : E2E : Connector V2 : Hudi + + 1.19.1 + 8.5.6 + + + + + org.testcontainers + minio + ${testcontainer.version} + test + + + io.minio + minio + ${minio.version} + test + + + + org.apache.seatunnel + connector-seatunnel-e2e-base + ${project.version} + tests + test-jar + test + + org.apache.seatunnel connector-hudi diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiIT.java index b0cafa65bc1..28f2eb3f530 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiIT.java @@ -20,12 +20,14 @@ import org.apache.seatunnel.common.utils.FileUtils; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.container.TestContainerId; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.example.data.Group; import org.apache.parquet.hadoop.ParquetReader; @@ -38,6 +40,7 @@ import lombok.extern.slf4j.Slf4j; +import java.io.File; import java.io.IOException; import java.net.URISyntaxException; import java.util.concurrent.TimeUnit; @@ -51,6 +54,9 @@ @Slf4j public class HudiIT extends TestSuiteBase { + private static final String DATABASE = "st"; + private static final String DEFAULT_DATABASE = "default"; + private static final String TABLE_NAME = "st_test"; private static final String TABLE_PATH = "/tmp/hudi/"; private static final String NAMESPACE = "hudi"; private static final String NAMESPACE_TAR = "hudi.tar.gz"; @@ -76,7 +82,6 @@ private void extractFiles() { "sh", "-c", "cd /tmp" + " && tar -zxvf " + NAMESPACE_TAR); try { Process process = processBuilder.start(); - // 等待命令执行完成 int exitCode = process.waitFor(); if (exitCode == 0) { log.info("Extract files successful."); @@ -97,12 +102,61 @@ private void extractFiles() { }; @TestTemplate + @DisabledOnContainer( + value = {TestContainerId.SPARK_2_4}, + type = {EngineType.FLINK}, + disabledReason = "FLINK do not support local file catalog in hudi.") public void testWriteHudi(TestContainer container) throws IOException, InterruptedException, URISyntaxException { Container.ExecResult textWriteResult = container.executeJob("/fake_to_hudi.conf"); Assertions.assertEquals(0, textWriteResult.getExitCode()); Configuration configuration = new Configuration(); - Path inputPath = new Path(TABLE_PATH); + configuration.set("fs.defaultFS", LocalFileSystem.DEFAULT_FS); + Path inputPath = + new Path(TABLE_PATH + File.separator + DATABASE + File.separator + TABLE_NAME); + + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy hudi to local + container.executeExtraCommands(containerExtendedFactory); + ParquetReader reader = + ParquetReader.builder(new GroupReadSupport(), inputPath) + .withConf(configuration) + .build(); + + long rowCount = 0; + + // Read data and count rows + while (reader.read() != null) { + rowCount++; + } + Assertions.assertEquals(5, rowCount); + }); + FileUtils.deleteFile(TABLE_PATH); + } + + @TestTemplate + @DisabledOnContainer( + value = {TestContainerId.SPARK_2_4}, + type = {EngineType.FLINK}, + disabledReason = "FLINK do not support local file catalog in hudi.") + public void testWriteHudiWithOmitConfigItem(TestContainer container) + throws IOException, InterruptedException, URISyntaxException { + Container.ExecResult textWriteResult = + container.executeJob("/fake_to_hudi_with_omit_config_item.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + Configuration configuration = new Configuration(); + configuration.set("fs.defaultFS", LocalFileSystem.DEFAULT_FS); + Path inputPath = + new Path( + TABLE_PATH + + File.separator + + DEFAULT_DATABASE + + File.separator + + TABLE_NAME); given().ignoreExceptions() .await() diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiMultiTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiMultiTableIT.java new file mode 100644 index 00000000000..0a9c4555ad2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiMultiTableIT.java @@ -0,0 +1,185 @@ +/* + * 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.hudi; + +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestContainerId; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; + +import lombok.extern.slf4j.Slf4j; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +import static org.awaitility.Awaitility.given; + +@Slf4j +public class HudiMultiTableIT extends TestSuiteBase { + + private static final String DATABASE_1 = "st1"; + private static final String TABLE_NAME_1 = "st_test_1"; + private static final String DATABASE_2 = "default"; + private static final String TABLE_NAME_2 = "st_test_2"; + private static final String TABLE_PATH = "/tmp/hudi/"; + private static final String NAMESPACE = "hudi"; + private static final String NAMESPACE_TAR = "hudi.tar.gz"; + + protected final ContainerExtendedFactory containerExtendedFactory = + new ContainerExtendedFactory() { + @Override + public void extend(GenericContainer container) + throws IOException, InterruptedException { + container.execInContainer( + "sh", + "-c", + "cd /tmp" + " && tar -czvf " + NAMESPACE_TAR + " " + NAMESPACE); + container.copyFileFromContainer( + "/tmp/" + NAMESPACE_TAR, "/tmp/" + NAMESPACE_TAR); + + extractFiles(); + } + + private void extractFiles() { + ProcessBuilder processBuilder = new ProcessBuilder(); + processBuilder.command( + "sh", "-c", "cd /tmp" + " && tar -zxvf " + NAMESPACE_TAR); + try { + Process process = processBuilder.start(); + int exitCode = process.waitFor(); + if (exitCode == 0) { + log.info("Extract files successful."); + } else { + log.error("Extract files failed with exit code " + exitCode); + } + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + }; + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + container.execInContainer("sh", "-c", "mkdir -p " + TABLE_PATH); + container.execInContainer("sh", "-c", "chmod -R 777 " + TABLE_PATH); + }; + + @TestTemplate + @DisabledOnContainer( + value = {TestContainerId.SPARK_2_4}, + type = {EngineType.FLINK}, + disabledReason = "FLINK do not support local file catalog in hudi.") + public void testMultiWrite(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult textWriteResult = container.executeJob("/multi_fake_to_hudi.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + Configuration configuration = new Configuration(); + configuration.set("fs.defaultFS", LocalFileSystem.DEFAULT_FS); + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy hudi to local + container.executeExtraCommands(containerExtendedFactory); + Path inputPath1 = + getNewestCommitFilePath( + new File( + TABLE_PATH + + File.separator + + DATABASE_1 + + File.separator + + TABLE_NAME_1)); + Path inputPath2 = + getNewestCommitFilePath( + new File( + TABLE_PATH + + File.separator + + DATABASE_2 + + File.separator + + TABLE_NAME_2)); + ParquetReader reader1 = + ParquetReader.builder(new GroupReadSupport(), inputPath1) + .withConf(configuration) + .build(); + ParquetReader reader2 = + ParquetReader.builder(new GroupReadSupport(), inputPath2) + .withConf(configuration) + .build(); + + long rowCount1 = 0; + long rowCount2 = 0; + // Read data and count rows + while (reader1.read() != null) { + rowCount1++; + } + // Read data and count rows + while (reader2.read() != null) { + rowCount2++; + } + Assertions.assertEquals(100, rowCount1); + Assertions.assertEquals(240, rowCount2); + }); + FileUtils.deleteFile(TABLE_PATH); + } + + public static Path getNewestCommitFilePath(File tablePathDir) throws IOException { + File[] files = FileUtil.listFiles(tablePathDir); + Long newestCommitTime = + Arrays.stream(files) + .filter(file -> file.getName().endsWith(".parquet")) + .map( + file -> + Long.parseLong( + file.getName() + .substring( + file.getName().lastIndexOf("_") + 1, + file.getName() + .lastIndexOf(".parquet")))) + .max(Long::compareTo) + .orElseThrow( + () -> + new IllegalArgumentException( + "Not found parquet file in " + tablePathDir)); + for (File file : files) { + if (file.getName().endsWith(newestCommitTime + ".parquet")) { + return new Path(file.toURI()); + } + } + throw new IllegalArgumentException("Not found parquet file in " + tablePathDir); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSeatunnelS3MultiTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSeatunnelS3MultiTableIT.java new file mode 100644 index 00000000000..67f3e9e884e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSeatunnelS3MultiTableIT.java @@ -0,0 +1,221 @@ +/* + * 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.hudi; + +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; +import org.apache.seatunnel.engine.e2e.SeaTunnelContainer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.MinIOContainer; + +import io.minio.BucketExistsArgs; +import io.minio.DownloadObjectArgs; +import io.minio.ListObjectsArgs; +import io.minio.MakeBucketArgs; +import io.minio.MinioClient; +import io.minio.Result; +import io.minio.messages.Item; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.concurrent.TimeUnit; + +import static org.awaitility.Awaitility.given; + +@Slf4j +public class HudiSeatunnelS3MultiTableIT extends SeaTunnelContainer { + + private static final String MINIO_DOCKER_IMAGE = "minio/minio:RELEASE.2024-06-13T22-53-53Z"; + private static final String HOST = "minio"; + private static final int MINIO_PORT = 9000; + private static final String MINIO_USER_NAME = "minio"; + private static final String MINIO_USER_PASSWORD = "miniominio"; + private static final String BUCKET = "hudi"; + + private MinIOContainer container; + private MinioClient minioClient; + + private static final String DATABASE_1 = "st1"; + private static final String TABLE_NAME_1 = "st_test_1"; + private static final String DATABASE_2 = "default"; + private static final String TABLE_NAME_2 = "st_test_2"; + private static final String DOWNLOAD_PATH = "/tmp/seatunnel/"; + + protected static final String AWS_SDK_DOWNLOAD = + "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.11.271/aws-java-sdk-bundle-1.11.271.jar"; + protected static final String HADOOP_AWS_DOWNLOAD = + "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.1.4/hadoop-aws-3.1.4.jar"; + + @Override + @BeforeAll + public void startUp() throws Exception { + container = + new MinIOContainer(MINIO_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withUserName(MINIO_USER_NAME) + .withPassword(MINIO_USER_PASSWORD) + .withExposedPorts(MINIO_PORT); + container.start(); + + String s3URL = container.getS3URL(); + + // configuringClient + minioClient = + MinioClient.builder() + .endpoint(s3URL) + .credentials(container.getUserName(), container.getPassword()) + .build(); + + // create bucket + minioClient.makeBucket(MakeBucketArgs.builder().bucket(BUCKET).build()); + + BucketExistsArgs existsArgs = BucketExistsArgs.builder().bucket(BUCKET).build(); + Assertions.assertTrue(minioClient.bucketExists(existsArgs)); + super.startUp(); + } + + @Override + protected String[] buildStartCommand() { + return new String[] { + "bash", + "-c", + "wget -P " + + SEATUNNEL_HOME + + "lib " + + AWS_SDK_DOWNLOAD + + " &&" + + "wget -P " + + SEATUNNEL_HOME + + "lib " + + HADOOP_AWS_DOWNLOAD + + " &&" + + ContainerUtil.adaptPathForWin( + Paths.get(SEATUNNEL_HOME, "bin", SERVER_SHELL).toString()) + }; + } + + @Override + @AfterAll + public void tearDown() throws Exception { + super.tearDown(); + if (container != null) { + container.close(); + } + } + + @Test + public void testS3MultiWrite() throws IOException, InterruptedException { + copyFileToContainer("/core-site.xml", "/tmp/seatunnel/config/core-site.xml"); + Container.ExecResult textWriteResult = executeSeaTunnelJob("/s3_fake_to_hudi.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + Configuration configuration = new Configuration(); + configuration.set("fs.defaultFS", LocalFileSystem.DEFAULT_FS); + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy hudi to local + Path inputPath1 = + downloadNewestCommitFile(DATABASE_1 + "/" + TABLE_NAME_1 + "/"); + Path inputPath2 = + downloadNewestCommitFile(DATABASE_2 + "/" + TABLE_NAME_2 + "/"); + ParquetReader reader1 = + ParquetReader.builder(new GroupReadSupport(), inputPath1) + .withConf(configuration) + .build(); + ParquetReader reader2 = + ParquetReader.builder(new GroupReadSupport(), inputPath2) + .withConf(configuration) + .build(); + + long rowCount1 = 0; + long rowCount2 = 0; + // Read data and count rows + while (reader1.read() != null) { + rowCount1++; + } + // Read data and count rows + while (reader2.read() != null) { + rowCount2++; + } + FileUtils.deleteFile(inputPath1.toUri().getPath()); + FileUtils.deleteFile(inputPath2.toUri().getPath()); + Assertions.assertEquals(100, rowCount1); + Assertions.assertEquals(240, rowCount2); + }); + } + + public Path downloadNewestCommitFile(String pathPrefix) throws IOException { + Iterable> listObjects = + minioClient.listObjects( + ListObjectsArgs.builder().bucket(BUCKET).prefix(pathPrefix).build()); + String newestCommitFileabsolutePath = ""; + String newestCommitFileName = ""; + long newestCommitTime = 0L; + for (Result listObject : listObjects) { + Item item; + try { + item = listObject.get(); + } catch (Exception e) { + throw new IOException("List minio file error.", e); + } + if (item.isDir() || !item.objectName().endsWith(".parquet")) { + continue; + } + long fileCommitTime = + Long.parseLong( + item.objectName() + .substring( + item.objectName().lastIndexOf("_") + 1, + item.objectName().lastIndexOf(".parquet"))); + if (fileCommitTime > newestCommitTime) { + newestCommitFileabsolutePath = item.objectName(); + newestCommitFileName = + newestCommitFileabsolutePath.substring( + item.objectName().lastIndexOf("/") + 1); + newestCommitTime = fileCommitTime; + } + } + try { + minioClient.downloadObject( + DownloadObjectArgs.builder() + .bucket(BUCKET) + .object(newestCommitFileabsolutePath) + .filename(DOWNLOAD_PATH + newestCommitFileName) + .build()); + } catch (Exception e) { + log.error("Download file from minio error."); + } + return new Path(DOWNLOAD_PATH + newestCommitFileName); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSparkS3MultiTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSparkS3MultiTableIT.java new file mode 100644 index 00000000000..db43348aefd --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hudi/HudiSparkS3MultiTableIT.java @@ -0,0 +1,202 @@ +/* + * 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.hudi; + +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestContainerId; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.MinIOContainer; + +import io.minio.BucketExistsArgs; +import io.minio.DownloadObjectArgs; +import io.minio.ListObjectsArgs; +import io.minio.MakeBucketArgs; +import io.minio.MinioClient; +import io.minio.Result; +import io.minio.messages.Item; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.awaitility.Awaitility.given; + +@Slf4j +public class HudiSparkS3MultiTableIT extends TestSuiteBase implements TestResource { + + private static final String MINIO_DOCKER_IMAGE = "minio/minio:RELEASE.2024-06-13T22-53-53Z"; + private static final String HOST = "minio"; + private static final int MINIO_PORT = 9000; + private static final String MINIO_USER_NAME = "minio"; + private static final String MINIO_USER_PASSWORD = "miniominio"; + private static final String BUCKET = "hudi"; + + private MinIOContainer container; + private MinioClient minioClient; + + private static final String DATABASE_1 = "st1"; + private static final String TABLE_NAME_1 = "st_test_1"; + private static final String DATABASE_2 = "default"; + private static final String TABLE_NAME_2 = "st_test_2"; + private static final String DOWNLOAD_PATH = "/tmp/seatunnel/"; + + @BeforeEach + @Override + public void startUp() throws Exception { + container = + new MinIOContainer(MINIO_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withUserName(MINIO_USER_NAME) + .withPassword(MINIO_USER_PASSWORD) + .withExposedPorts(MINIO_PORT); + container.start(); + + String s3URL = container.getS3URL(); + + // configuringClient + minioClient = + MinioClient.builder() + .endpoint(s3URL) + .credentials(container.getUserName(), container.getPassword()) + .build(); + + // create bucket + minioClient.makeBucket(MakeBucketArgs.builder().bucket(BUCKET).build()); + + BucketExistsArgs existsArgs = BucketExistsArgs.builder().bucket(BUCKET).build(); + Assertions.assertTrue(minioClient.bucketExists(existsArgs)); + } + + @AfterEach + @Override + public void tearDown() throws Exception { + if (container != null) { + container.close(); + } + } + + @TestTemplate + @DisabledOnContainer( + value = {TestContainerId.SPARK_2_4}, + type = {EngineType.FLINK, EngineType.SEATUNNEL}, + disabledReason = + "The hadoop version in current flink image is not compatible with the aws version and default container of seatunnel not support s3.") + public void testS3MultiWrite(TestContainer container) throws IOException, InterruptedException { + container.copyFileToContainer("/core-site.xml", "/tmp/seatunnel/config/core-site.xml"); + Container.ExecResult textWriteResult = container.executeJob("/s3_fake_to_hudi.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + Configuration configuration = new Configuration(); + configuration.set("fs.defaultFS", LocalFileSystem.DEFAULT_FS); + given().ignoreExceptions() + .await() + .atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + // copy hudi to local + Path inputPath1 = + downloadNewestCommitFile(DATABASE_1 + "/" + TABLE_NAME_1 + "/"); + Path inputPath2 = + downloadNewestCommitFile(DATABASE_2 + "/" + TABLE_NAME_2 + "/"); + ParquetReader reader1 = + ParquetReader.builder(new GroupReadSupport(), inputPath1) + .withConf(configuration) + .build(); + ParquetReader reader2 = + ParquetReader.builder(new GroupReadSupport(), inputPath2) + .withConf(configuration) + .build(); + + long rowCount1 = 0; + long rowCount2 = 0; + // Read data and count rows + while (reader1.read() != null) { + rowCount1++; + } + // Read data and count rows + while (reader2.read() != null) { + rowCount2++; + } + FileUtils.deleteFile(inputPath1.toUri().getPath()); + FileUtils.deleteFile(inputPath2.toUri().getPath()); + Assertions.assertEquals(100, rowCount1); + Assertions.assertEquals(240, rowCount2); + }); + } + + public Path downloadNewestCommitFile(String pathPrefix) throws IOException { + Iterable> listObjects = + minioClient.listObjects( + ListObjectsArgs.builder().bucket(BUCKET).prefix(pathPrefix).build()); + String newestCommitFileabsolutePath = ""; + String newestCommitFileName = ""; + long newestCommitTime = 0L; + for (Result listObject : listObjects) { + Item item; + try { + item = listObject.get(); + } catch (Exception e) { + throw new IOException("List minio file error.", e); + } + if (item.isDir() || !item.objectName().endsWith(".parquet")) { + continue; + } + long fileCommitTime = + Long.parseLong( + item.objectName() + .substring( + item.objectName().lastIndexOf("_") + 1, + item.objectName().lastIndexOf(".parquet"))); + if (fileCommitTime > newestCommitTime) { + newestCommitFileabsolutePath = item.objectName(); + newestCommitFileName = + newestCommitFileabsolutePath.substring( + item.objectName().lastIndexOf("/") + 1); + newestCommitTime = fileCommitTime; + } + } + try { + minioClient.downloadObject( + DownloadObjectArgs.builder() + .bucket(BUCKET) + .object(newestCommitFileabsolutePath) + .filename(DOWNLOAD_PATH + newestCommitFileName) + .build()); + } catch (Exception e) { + log.error("Download file from minio error."); + } + return new Path(DOWNLOAD_PATH + newestCommitFileName); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/core-site.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/core-site.xml new file mode 100644 index 00000000000..9a0f3431692 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/core-site.xml @@ -0,0 +1,57 @@ + + + + + + + + fs.defaultFS + s3a://hudi + true + + + fs.s3a.access.key + minio + + + fs.s3a.secret.key + miniominio + + + fs.s3a.endpoint + http://minio:9000 + + + fs.s3a.path.style.access + true + + + fs.s3a.aws.credentials.provider + org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider + + + fs.s3a.connection.ssl.enabled + false + + + fs.s3a.impl.disable.cache + true + + + fs.s3a.threads.keepalivetime + 10 + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi.conf index a02bb0fc726..894f1191149 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi.conf @@ -46,7 +46,11 @@ source { sink { Hudi { + op_type="UPSERT" table_dfs_path = "/tmp/hudi" + database = "st" table_name = "st_test" + table_type="COPY_ON_WRITE" + record_key_fields="c_bigint" } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi_with_omit_config_item.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi_with_omit_config_item.conf new file mode 100644 index 00000000000..a02bb0fc726 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/fake_to_hudi_with_omit_config_item.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + 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_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + result_table_name = "fake" + } +} + +sink { + Hudi { + table_dfs_path = "/tmp/hudi" + table_name = "st_test" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/multi_fake_to_hudi.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/multi_fake_to_hudi.conf new file mode 100644 index 00000000000..517131331b6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/multi_fake_to_hudi.conf @@ -0,0 +1,93 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + row.num = 100 + schema = { + table = "st_test_1" + 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_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + }, + { + row.num = 240 + schema = { + table = "st_test_2" + 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_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + } + ] + } +} + +sink { + Hudi { + auto_commit = "false" + table_dfs_path = "/tmp/hudi" + table_list=[ + { + database = "st1" + table_name = "st_test_1" + table_type="COPY_ON_WRITE" + record_key_fields="c_bigint" + op_type="UPSERT" + batch_size = 100 + }, + { + table_name = "st_test_2" + batch_size = 100 + } + ] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/s3_fake_to_hudi.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/s3_fake_to_hudi.conf new file mode 100644 index 00000000000..0434950454a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hudi-e2e/src/test/resources/s3_fake_to_hudi.conf @@ -0,0 +1,94 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + row.num = 100 + schema = { + table = "st_test_1" + 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_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + }, + { + row.num = 240 + schema = { + table = "st_test_2" + 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_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + } + ] + } +} + +sink { + Hudi { + conf_files_path = "/tmp/seatunnel/config/core-site.xml" + auto_commit = "false" + table_dfs_path = "s3a://hudi/" + table_list=[ + { + database = "st1" + table_name = "st_test_1" + table_type="COPY_ON_WRITE" + op_type="UPSERT" + record_key_fields="c_bigint" + batch_size = 100 + }, + { + table_name = "st_test_2" + batch_size = 100 + } + ] + } +} diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java index 4a30756f8f7..96e5162d7a9 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java @@ -86,9 +86,7 @@ private GenericContainer createSeaTunnelServer() throws IOException, Interrup new GenericContainer<>(getDockerImage()) .withNetwork(NETWORK) .withEnv("TZ", "UTC") - .withCommand( - ContainerUtil.adaptPathForWin( - Paths.get(SEATUNNEL_HOME, "bin", SERVER_SHELL).toString())) + .withCommand(buildStartCommand()) .withNetworkAliases("server") .withExposedPorts() .withLogConsumer( @@ -117,6 +115,12 @@ private GenericContainer createSeaTunnelServer() throws IOException, Interrup return server; } + protected String[] buildStartCommand() { + return new String[] { + ContainerUtil.adaptPathForWin(Paths.get(SEATUNNEL_HOME, "bin", SERVER_SHELL).toString()) + }; + } + protected GenericContainer createSeaTunnelContainerWithFakeSourceAndInMemorySink( String configFilePath) throws IOException, InterruptedException { GenericContainer server = @@ -372,7 +376,8 @@ private static boolean isSystemThread(String s) { // The renewed background thread of the hdfs client || s.startsWith("LeaseRenewer") // The read of hdfs which has the thread that is all in running status - || s.startsWith("org.apache.hadoop.hdfs.PeerCache"); + || s.startsWith("org.apache.hadoop.hdfs.PeerCache") + || s.startsWith("java-sdk-progress-listener-callback-thread"); } private void classLoaderObjectCheck(Integer maxSize) throws IOException, InterruptedException { diff --git a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml index 322c1ed65b5..be5ced9214a 100644 --- a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml +++ b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml @@ -97,7 +97,6 @@ com.google.common.base.* com.google.common.cache.* com.google.common.collect.* - com.google.common.util.* From 4f5d27f625b4b1964b66b3ede76b4e47b4196149 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 19 Sep 2024 21:22:43 +0800 Subject: [PATCH 7/7] [Fix][Connector-V2] Fix jdbc test case failed (#7690) --- .../jdbc/config/JdbcSourceTableConfig.java | 4 +++- .../connectors/seatunnel/jdbc/AbstractJdbcIT.java | 15 +++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java index f0af8513776..a3522e9c146 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java @@ -101,7 +101,9 @@ public static List of(ReadonlyConfig connectorConfig) { if (tableList.size() > 1) { List tableIds = - tableList.stream().map(e -> e.getTablePath()).collect(Collectors.toList()); + tableList.stream() + .map(JdbcSourceTableConfig::getTablePath) + .collect(Collectors.toList()); Set tableIdSet = new HashSet<>(tableIds); if (tableIdSet.size() < tableList.size() - 1) { throw new IllegalArgumentException( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 6b574512c76..24b916d4049 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; @@ -463,19 +464,17 @@ public void testCatalog() { catalog.dropDatabase(targetTablePath, false); Assertions.assertFalse(catalog.databaseExists(targetTablePath.getDatabaseName())); } - - TableNotExistException exception = + Exception exception = Assertions.assertThrows( - TableNotExistException.class, + Exception.class, () -> catalog.truncateTable( TablePath.of("not_exist", "not_exist", "not_exist"), false)); - Assertions.assertEquals( - String.format( - "ErrorCode:[API-05], ErrorDescription:[Table not existed] - Table not_exist.not_exist.not_exist does not exist in Catalog %s.", - catalog.name()), - exception.getMessage()); + + Assertions.assertTrue( + exception instanceof TableNotExistException + || exception instanceof CatalogException); } @Test