diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index c57dbcb629e..b417d53e72a 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -257,11 +257,19 @@ activemq: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-activemq/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(activemq)/**' + qdrant: - all: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-qdrant/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(qdrant)/**' + +typesense: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-typesense/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(typesense)/**' + Zeta Rest API: - changed-files: - any-glob-to-any-file: seatunnel-engine/**/server/rest/** diff --git a/config/plugin_config b/config/plugin_config index c06829376ba..26eb4cab4a6 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -89,5 +89,5 @@ connector-milvus connector-activemq connector-sls connector-qdrant +connector-typesense connector-cdc-opengauss ---end-- diff --git a/docs/en/connector-v2/sink/Rabbitmq.md b/docs/en/connector-v2/sink/Rabbitmq.md index 489287249e0..c7963525fba 100644 --- a/docs/en/connector-v2/sink/Rabbitmq.md +++ b/docs/en/connector-v2/sink/Rabbitmq.md @@ -57,6 +57,21 @@ convenience method for setting the fields in an AMQP URI: host, port, username, the queue to write the message to +### durable [boolean] + +true: The queue will survive a server restart. +false: The queue will be deleted on server restart. + +### exclusive [boolean] + +true: The queue is used only by the current connection and will be deleted when the connection closes. +false: The queue can be used by multiple connections. + +### auto_delete [boolean] + +true: The queue will be deleted automatically when the last consumer unsubscribes. +false: The queue will not be automatically deleted. + ### schema [Config] #### fields [Config] @@ -112,6 +127,30 @@ sink { } ``` +### Example 2 + +queue with durable, exclusive, auto_delete: + +```hocon +sink { + RabbitMQ { + host = "rabbitmq-e2e" + port = 5672 + virtual_host = "/" + username = "guest" + password = "guest" + queue_name = "test1" + durable = "true" + exclusive = "false" + auto_delete = "false" + rabbitmq.config = { + requested-heartbeat = 10 + connection-timeout = 10 + } + } +} +``` + ## Changelog ### next version diff --git a/docs/en/connector-v2/sink/Typesense.md b/docs/en/connector-v2/sink/Typesense.md new file mode 100644 index 00000000000..8700d68dc77 --- /dev/null +++ b/docs/en/connector-v2/sink/Typesense.md @@ -0,0 +1,93 @@ +# Typesense + +## Description + +Outputs data to `Typesense`. + +## Key Features + +- [ ] [Exactly Once](../../concept/connector-v2-features.md) +- [x] [CDC](../../concept/connector-v2-features.md) + +## Options + +| Name | Type | Required | Default Value | +|------------------|--------|----------|------------------------------| +| hosts | array | Yes | - | +| collection | string | Yes | - | +| schema_save_mode | string | Yes | CREATE_SCHEMA_WHEN_NOT_EXIST | +| data_save_mode | string | Yes | APPEND_DATA | +| primary_keys | array | No | | +| key_delimiter | string | No | `_` | +| api_key | string | No | | +| max_retry_count | int | No | 3 | +| max_batch_size | int | No | 10 | +| common-options | | No | - | + +### hosts [array] + +The access address for Typesense, formatted as `host:port`, e.g., `["typesense-01:8108"]`. + +### collection [string] + +The name of the collection to write to, e.g., "seatunnel". + +### primary_keys [array] + +Primary key fields used to generate the document `id`. + +### key_delimiter [string] + +Sets the delimiter for composite keys (default is `_`). + +### api_key [config] + +The `api_key` for secure access to Typesense. + +### max_retry_count [int] + +The maximum number of retry attempts for batch requests. + +### max_batch_size [int] + +The maximum size of document batches. + +### common options + +Common parameters for Sink plugins. Refer to [Common Sink Options](../source-common-options.md) for more details. + +### schema_save_mode + +Choose how to handle the target-side schema before starting the synchronization task: +- `RECREATE_SCHEMA`: Creates the table if it doesn’t exist, and deletes and recreates it if it does. +- `CREATE_SCHEMA_WHEN_NOT_EXIST`: Creates the table if it doesn’t exist, skips creation if it does. +- `ERROR_WHEN_SCHEMA_NOT_EXIST`: Throws an error if the table doesn’t exist. + +### data_save_mode + +Choose how to handle existing data on the target side before starting the synchronization task: +- `DROP_DATA`: Retains the database structure but deletes the data. +- `APPEND_DATA`: Retains both the database structure and the data. +- `ERROR_WHEN_DATA_EXISTS`: Throws an error if data exists. + +## Example + +Simple example: + +```bash +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["localhost:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} +``` + diff --git a/docs/en/connector-v2/source/Typesense.md b/docs/en/connector-v2/source/Typesense.md new file mode 100644 index 00000000000..9f88a58d4a5 --- /dev/null +++ b/docs/en/connector-v2/source/Typesense.md @@ -0,0 +1,79 @@ +# Typesense + +> Typesense Source Connector + +## Description + +Reads data from Typesense. + +## Key Features + +- [x] [Batch Processing](../../concept/connector-v2-features.md) +- [ ] [Stream Processing](../../concept/connector-v2-features.md) +- [ ] [Exactly-Once](../../concept/connector-v2-features.md) +- [x] [Schema](../../concept/connector-v2-features.md) +- [x] [Parallelism](../../concept/connector-v2-features.md) +- [ ] [User-Defined Splits Support](../../concept/connector-v2-features.md) + +## Options + +| Name | Type | Required | Default | +|------------|--------|----------|---------| +| hosts | array | yes | - | +| collection | string | yes | - | +| schema | config | yes | - | +| api_key | string | no | - | +| query | string | no | - | +| batch_size | int | no | 100 | + +### hosts [array] + +The access address of Typesense, for example: `["typesense-01:8108"]`. + +### collection [string] + +The name of the collection to write to, for example: `"seatunnel"`. + +### schema [config] + +The columns to be read from Typesense. For more information, please refer to the [guide](../../concept/schema-feature.md#how-to-declare-type-supported). + +### api_key [config] + +The `api_key` for Typesense security authentication. + +### batch_size + +The number of records to query per batch when reading data. + +### Common Options + +For common parameters of Source plugins, please refer to [Source Common Options](../source-common-options.md). + +## Example + +```bash +source { + Typesense { + hosts = ["localhost:8108"] + collection = "companies" + api_key = "xyz" + query = "q=*&filter_by=num_employees:>9000" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} +``` + diff --git a/docs/en/faq.md b/docs/en/faq.md index 2e50c9d4618..02c125ad4fd 100644 --- a/docs/en/faq.md +++ b/docs/en/faq.md @@ -203,23 +203,6 @@ spark { } ``` -## How do I specify a different JDK version for SeaTunnel on YARN? - -For example, if you want to set the JDK version to JDK8, there are two cases: - -- The YARN cluster has deployed JDK8, but the default JDK is not JDK8. Add two configurations to the SeaTunnel config file: - - ``` - env { - ... - spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" - spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" - ... - } - ``` -- YARN cluster does not deploy JDK8. At this time, start SeaTunnel attached with JDK8. For detailed operations, see: - https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html - ## What should I do if OOM always appears when running SeaTunnel in Spark local[*] mode? If you run in local mode, you need to modify the `start-seatunnel.sh` startup script. After `spark-submit`, add a parameter `--driver-memory 4g` . Under normal circumstances, local mode is not used in the production environment. Therefore, this parameter generally does not need to be set during On YARN. See: [Application Properties](https://spark.apache.org/docs/latest/configuration.html#application-properties) for details. @@ -334,10 +317,6 @@ spark-submit --verbose ... ``` -## How do I use SeaTunnel to synchronize data across HDFS clusters? - -Just configure hdfs-site.xml properly. Refer to: https://www.cnblogs.com/suanec/p/7828139.html. - ## I want to learn the source code of SeaTunnel. Where should I start? SeaTunnel has a completely abstract and structured code implementation, and many people have chosen SeaTunnel As a way to learn Spark. You can learn the source code from the main program entry: SeaTunnel.java diff --git a/docs/zh/connector-v2/sink/Typesense.md b/docs/zh/connector-v2/sink/Typesense.md new file mode 100644 index 00000000000..99017f32cb5 --- /dev/null +++ b/docs/zh/connector-v2/sink/Typesense.md @@ -0,0 +1,95 @@ +# Typesense + +## 描述 + +输出数据到 `Typesense` + +## 主要特性 + +- [ ] [精确一次](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------|--------|------|------------------------------| +| hosts | array | 是 | - | +| collection | string | 是 | - | +| schema_save_mode | string | 是 | CREATE_SCHEMA_WHEN_NOT_EXIST | +| data_save_mode | string | 是 | APPEND_DATA | +| primary_keys | array | 否 | | +| key_delimiter | string | 否 | `_` | +| api_key | string | 否 | | +| max_retry_count | int | 否 | 3 | +| max_batch_size | int | 否 | 10 | +| common-options | | 否 | - | + +### hosts [array] + +Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] + +### collection [string] + +要写入的集合名,例如:“seatunnel” + +### primary_keys [array] + +主键字段用于生成文档 `id`。 + +### key_delimiter [string] + +设定复合键的分隔符(默认为 `_`)。 + +### api_key [config] + +typesense 安全认证的 api_key。 + +### max_retry_count [int] + +批次批量请求最大尝试大小 + +### max_batch_size [int] + +批次批量文档最大大小 + +### common options + +Sink插件常用参数,请参考 [Sink常用选项](../sink-common-options.md) 了解详情 + +### schema_save_mode + +在启动同步任务之前,针对目标侧已有的表结构选择不同的处理方案
+选项介绍:
+`RECREATE_SCHEMA` :当表不存在时会创建,当表已存在时会删除并重建
+`CREATE_SCHEMA_WHEN_NOT_EXIST` :当表不存在时会创建,当表已存在时则跳过创建
+`ERROR_WHEN_SCHEMA_NOT_EXIST` :当表不存在时将抛出错误
+ +### data_save_mode + +在启动同步任务之前,针对目标侧已存在的数据选择不同的处理方案
+选项介绍:
+`DROP_DATA`: 保留数据库结构,删除数据
+`APPEND_DATA`:保留数据库结构,保留数据
+`ERROR_WHEN_DATA_EXISTS`:当有数据时抛出错误
+ +## 示例 + +简单示例 + +```bash +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["localhost:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} +``` + diff --git a/docs/zh/connector-v2/source/Typesense.md b/docs/zh/connector-v2/source/Typesense.md new file mode 100644 index 00000000000..35f04e23a27 --- /dev/null +++ b/docs/zh/connector-v2/source/Typesense.md @@ -0,0 +1,79 @@ +# Typesense + +> Typesense 源连接器 + +## 描述 + +从 Typesense 读取数据。 + +## 主要功能 + +- [x] [批处理](../../concept/connector-v2-features.md) +- [ ] [流处理](../../concept/connector-v2-features.md) +- [ ] [精确一次](../../concept/connector-v2-features.md) +- [x] [Schema](../../concept/connector-v2-features.md) +- [x] [并行度](../../concept/connector-v2-features.md) +- [ ] [支持用户定义的拆分](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|------------|--------|----|-----| +| hosts | array | 是 | - | +| collection | string | 是 | - | +| schema | config | 是 | - | +| api_key | string | 否 | - | +| query | string | 否 | - | +| batch_size | int | 否 | 100 | + +### hosts [array] + +Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] + +### collection [string] + +要写入的集合名,例如:“seatunnel” + +### schema [config] + +typesense 需要读取的列。有关更多信息,请参阅:[guide](../../concept/schema-feature.md#how-to-declare-type-supported)。 + +### api_key [config] + +typesense 安全认证的 api_key。 + +### batch_size + +读取数据时,每批次查询数量 + +### 常用选项 + +Source 插件常用参数,具体请参考 [Source 常用选项](../source-common-options.md) + +## 示例 + +```bash +source { + Typesense { + hosts = ["localhost:8108"] + collection = "companies" + api_key = "xyz" + query = "q=*&filter_by=num_employees:>9000" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} +``` + diff --git a/docs/zh/faq.md b/docs/zh/faq.md index 3be6ce38e56..4fc24e6a3ad 100644 --- a/docs/zh/faq.md +++ b/docs/zh/faq.md @@ -204,23 +204,6 @@ spark { } ``` -## 如何为 YARN 上的 SeaTunnel 指定不同的 JDK 版本? - -例如要设置JDK版本为JDK8,有两种情况: - -- YARN集群已部署JDK8,但默认JDK不是JDK8。 在 SeaTunnel 配置文件中添加两个配置: - - ``` - env { - ... - spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" - spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" - ... - } - ``` -- YARN集群未部署JDK8。 此时,启动附带JDK8的SeaTunnel。 详细操作参见: - https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html - ## Spark local[*]模式运行SeaTunnel时总是出现OOM怎么办? 如果以本地模式运行,则需要修改`start-seatunnel.sh`启动脚本。 在 `spark-submit` 之后添加参数 `--driver-memory 4g` 。 一般情况下,生产环境中不使用本地模式。 因此,On YARN时一般不需要设置该参数。 有关详细信息,请参阅:[应用程序属性](https://spark.apache.org/docs/latest/configuration.html#application-properties)。 @@ -335,10 +318,6 @@ spark-submit --verbose ... ``` -## 如何使用SeaTunnel跨HDFS集群同步数据? - -只需正确配置 hdfs-site.xml 即可。 参考:https://www.cnblogs.com/suanec/p/7828139.html。 - ## 我想学习SeaTunnel的源代码。 我应该从哪里开始? SeaTunnel 拥有完全抽象、结构化的代码实现,很多人都选择 SeaTunnel 作为学习 Spark 的方式。 您可以从主程序入口了解源代码:SeaTunnel.java diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 738fd93ba8b..1ddedd5ea8d 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -134,8 +134,9 @@ seatunnel.sink.ActiveMQ = connector-activemq seatunnel.source.Qdrant = connector-qdrant seatunnel.sink.Qdrant = connector-qdrant seatunnel.source.Sls = connector-sls +seatunnel.source.Typesense = connector-typesense +seatunnel.sink.Typesense = connector-typesense seatunnel.source.Opengauss-CDC = connector-cdc-opengauss - seatunnel.transform.Sql = seatunnel-transforms-v2 seatunnel.transform.FieldMapper = seatunnel-transforms-v2 seatunnel.transform.Filter = seatunnel-transforms-v2 diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java index 0f4610783af..bbb590502cf 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java @@ -50,25 +50,20 @@ public class IcebergCatalogLoader implements Serializable { private static final long serialVersionUID = -6003040601422350869L; private static final List HADOOP_CONF_FILES = ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - private CommonConfig config; + private final CommonConfig config; public IcebergCatalogLoader(CommonConfig config) { this.config = config; } public Catalog loadCatalog() { - // When using the seatunel engine, set the current class loader to prevent loading failures + // When using the SeaTunnel engine, set the current class loader to prevent loading failures Thread.currentThread().setContextClassLoader(IcebergCatalogLoader.class.getClassLoader()); return CatalogUtil.buildIcebergCatalog( config.getCatalogName(), config.getCatalogProps(), loadHadoopConfig(config)); } - /** - * Loading Hadoop configuration through reflection - * - * @param config - * @return - */ + /** Loading Hadoop configuration through reflection */ public Object loadHadoopConfig(CommonConfig config) { Class configClass = DynClasses.builder() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java index 520f9bdbac9..fc28001b2ca 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java @@ -58,9 +58,9 @@ @Slf4j public class IcebergCatalog implements Catalog { - private String catalogName; - private ReadonlyConfig readonlyConfig; - private IcebergCatalogLoader icebergCatalogLoader; + private final String catalogName; + private final ReadonlyConfig readonlyConfig; + private final IcebergCatalogLoader icebergCatalogLoader; private org.apache.iceberg.catalog.Catalog catalog; public IcebergCatalog(String catalogName, ReadonlyConfig readonlyConfig) { @@ -224,22 +224,21 @@ public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) { List columns = icebergTable.schema().columns(); TableSchema.Builder builder = TableSchema.builder(); - columns.stream() - .forEach( - nestedField -> { - String name = nestedField.name(); - SeaTunnelDataType seaTunnelType = - SchemaUtils.toSeaTunnelType(name, nestedField.type()); - PhysicalColumn physicalColumn = - PhysicalColumn.of( - name, - seaTunnelType, - (Long) null, - true, - null, - nestedField.doc()); - builder.column(physicalColumn); - }); + columns.forEach( + nestedField -> { + String name = nestedField.name(); + SeaTunnelDataType seaTunnelType = + SchemaUtils.toSeaTunnelType(name, nestedField.type()); + PhysicalColumn physicalColumn = + PhysicalColumn.of( + name, + seaTunnelType, + (Long) null, + true, + null, + nestedField.doc()); + builder.column(physicalColumn); + }); List partitionKeys = icebergTable.spec().fields().stream() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java index 8c699b34402..f46928456fb 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java @@ -92,17 +92,17 @@ private NameMapping createNameMapping(Table table) { return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; } - public Record convert(Object row, SeaTunnelDataType rowType) { + public Record convert(Object row, SeaTunnelDataType rowType) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, null); } - public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { + public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, wrapper); } protected GenericRecord convertStructValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.StructType schema, int parentFieldId, SchemaChangeWrapper wrapper) { @@ -120,15 +120,7 @@ protected GenericRecord convertStructValue( } } - /** - * Convert RowType - * - * @param row - * @param fromType - * @param schema - * @param structFieldId - * @return - */ + /** Convert RowType */ private GenericRecord convertToStruct( SeaTunnelRow row, SeaTunnelRowType fromType, @@ -179,7 +171,7 @@ private GenericRecord convertToStruct( public Object convertValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Type type, int fieldId, SchemaChangeWrapper wrapper) { @@ -252,7 +244,7 @@ private Map createStructNameMap(Types.StructType sche protected List convertListValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.ListType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value.getClass().isArray()); @@ -269,7 +261,7 @@ protected List convertListValue( protected Map convertMapValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.MapType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value instanceof Map); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java index 65bccbdb893..a1d43d6acfd 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java @@ -61,9 +61,9 @@ public class IcebergSink SupportSaveMode, SupportMultiTableSink { private static String PLUGIN_NAME = "Iceberg"; - private SinkConfig config; - private ReadonlyConfig readonlyConfig; - private CatalogTable catalogTable; + private final SinkConfig config; + private final ReadonlyConfig readonlyConfig; + private final CatalogTable catalogTable; public IcebergSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { this.readonlyConfig = pluginConfig; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java index aed6522ca87..3a5e22b93b4 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java @@ -54,13 +54,12 @@ public class IcebergSinkWriter implements SinkWriter, SupportMultiTableSinkWriter { private SeaTunnelRowType rowType; - private SinkConfig config; - private IcebergTableLoader icebergTableLoader; + private final SinkConfig config; + private final IcebergTableLoader icebergTableLoader; private RecordWriter writer; - private IcebergFilesCommitter filesCommitter; - private List results = Lists.newArrayList(); + private final IcebergFilesCommitter filesCommitter; + private final List results = Lists.newArrayList(); private String commitUser = UUID.randomUUID().toString(); - private long checkpointId; private final DataTypeChangeEventHandler dataTypeChangeEventHandler; @@ -77,7 +76,6 @@ public IcebergSinkWriter( tryCreateRecordWriter(); if (Objects.nonNull(states) && !states.isEmpty()) { this.commitUser = states.get(0).getCommitUser(); - this.checkpointId = states.get(0).getCheckpointId(); preCommit(states); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java index 2be206ebb6e..06b48591df1 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java @@ -54,7 +54,7 @@ public class IcebergRecordWriter implements RecordWriter { private final List writerResults; private TaskWriter writer; private RowConverter recordConverter; - private IcebergWriterFactory writerFactory; + private final IcebergWriterFactory writerFactory; public IcebergRecordWriter(Table table, IcebergWriterFactory writerFactory, SinkConfig config) { this.config = config; @@ -122,12 +122,7 @@ private void changeColumn( } } - /** - * apply schema update - * - * @param updates - * @return - */ + /** apply schema update */ private void applySchemaUpdate(SchemaChangeWrapper updates) { // complete the current file flush(); @@ -169,7 +164,4 @@ private void flush() { table.spec().partitionType())); writer = null; } - - @Override - public void close() {} } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java index 67809088ef7..2ee7c3d6d74 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java @@ -40,9 +40,6 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import lombok.extern.slf4j.Slf4j; import java.util.List; @@ -58,7 +55,6 @@ @Slf4j public class IcebergWriterFactory { - private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final IcebergTableLoader tableLoader; private final SinkConfig config; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java index 7a2fdf9d4ff..c56f3f2f00e 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -65,12 +65,12 @@ public class IcebergSource private static final long serialVersionUID = 4343414808223919870L; - private SourceConfig sourceConfig; - private Schema tableSchema; - private Schema projectedSchema; - private SeaTunnelRowType seaTunnelRowType; + private final SourceConfig sourceConfig; + private final Schema tableSchema; + private final Schema projectedSchema; + private final SeaTunnelRowType seaTunnelRowType; private JobContext jobContext; - private CatalogTable catalogTable; + private final CatalogTable catalogTable; public IcebergSource(ReadonlyConfig config, CatalogTable catalogTable) { this.sourceConfig = SourceConfig.loadConfig(config); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java index 6c99eb409c1..01343a119f6 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java @@ -40,7 +40,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -106,21 +105,11 @@ public static Table autoCreateTable( SinkConfig config = new SinkConfig(readonlyConfig); // build auto create table Map options = new HashMap<>(table.getOptions()); - options.put(TableProperties.FORMAT_VERSION, "2"); // override options.putAll(config.getAutoCreateProps()); return createTable(catalog, toIcebergTableIdentifier(tablePath), config, schema, options); } - /** - * For local test - * - * @param catalog - * @param tableIdentifier - * @param config - * @param rowType - * @return - */ public static Table autoCreateTable( Catalog catalog, TableIdentifier tableIdentifier, @@ -180,7 +169,7 @@ private static Table createTable( Optional pkId = structType.fields().stream() .filter(nestedField -> nestedField.name().equals(pk)) - .map(nestedField -> nestedField.fieldId()) + .map(Types.NestedField::fieldId) .findFirst(); if (!pkId.isPresent()) { throw new IllegalArgumentException( @@ -196,23 +185,14 @@ private static Table createTable( structType .fields() .forEach( - field -> { - fields.add( - identifierFieldIds.contains(field.fieldId()) - ? field.asRequired() - : field.asOptional()); - }); + field -> + fields.add( + identifierFieldIds.contains(field.fieldId()) + ? field.asRequired() + : field.asOptional())); return new Schema(fields, identifierFieldIds); } - public static TableIdentifier toIcebergTableIdentifierFromCatalogTable( - CatalogTable catalogTable) { - org.apache.seatunnel.api.table.catalog.TableIdentifier tableIdentifier = - catalogTable.getTableId(); - return TableIdentifier.of( - tableIdentifier.getDatabaseName(), tableIdentifier.getTableName()); - } - public static TableIdentifier toIcebergTableIdentifier(TablePath tablePath) { return TableIdentifier.of(tablePath.getDatabaseName(), tablePath.getTableName()); } @@ -221,12 +201,7 @@ public static TablePath toTablePath(TableIdentifier tableIdentifier) { return TablePath.of(tableIdentifier.namespace().toString(), tableIdentifier.name()); } - /** - * Commit table schema updates - * - * @param table - * @param wrapper - */ + /** Commit table schema updates */ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper) { // get the latest schema in case another process updated it table.refresh(); @@ -249,7 +224,7 @@ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper .collect(toList()); // Rename column name - List changeColumns = wrapper.changeColumns().stream().collect(toList()); + List changeColumns = new ArrayList<>(wrapper.changeColumns()); if (addColumns.isEmpty() && modifyColumns.isEmpty() @@ -294,7 +269,7 @@ public static SeaTunnelDataType toSeaTunnelType(String fieldName, Type type) return IcebergTypeMapper.mapping(fieldName, type); } - public static Type toIcebergType(SeaTunnelDataType rowType) { + public static Type toIcebergType(SeaTunnelDataType rowType) { return IcebergTypeMapper.toIcebergType(rowType); } diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java index 82ae2728d67..3f5c862cadf 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java @@ -189,11 +189,16 @@ public void close() { protected void setupQueue() throws IOException { if (config.getQueueName() != null) { - declareQueueDefaults(channel, config.getQueueName()); + declareQueueDefaults(channel, config); } } - private void declareQueueDefaults(Channel channel, String queueName) throws IOException { - channel.queueDeclare(queueName, true, false, false, null); + private void declareQueueDefaults(Channel channel, RabbitmqConfig config) throws IOException { + channel.queueDeclare( + config.getQueueName(), + config.getDurable(), + config.getExclusive(), + config.getAutoDelete(), + null); } } diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java index e8e2ce55c30..8475817457a 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java @@ -53,6 +53,9 @@ public class RabbitmqConfig implements Serializable { private Integer prefetchCount; private long deliveryTimeout; private String queueName; + private Boolean durable; + private Boolean exclusive; + private Boolean autoDelete; private String routingKey; private boolean logFailuresOnly = false; private String exchange = ""; @@ -195,6 +198,30 @@ public class RabbitmqConfig implements Serializable { "Whether the messages received are supplied with a unique" + "id to deduplicate messages (in case of failed acknowledgments)."); + public static final Option DURABLE = + Options.key("durable") + .booleanType() + .defaultValue(true) + .withDescription( + "true: The queue will survive a server restart." + + "false: The queue will be deleted on server restart."); + + public static final Option EXCLUSIVE = + Options.key("exclusive") + .booleanType() + .defaultValue(false) + .withDescription( + "true: The queue is used only by the current connection and will be deleted when the connection closes." + + "false: The queue can be used by multiple connections."); + + public static final Option AUTO_DELETE = + Options.key("auto_delete") + .booleanType() + .defaultValue(false) + .withDescription( + "true: The queue will be deleted automatically when the last consumer unsubscribes." + + "false: The queue will not be automatically deleted."); + private void parseSinkOptionProperties(Config pluginConfig) { if (CheckConfigUtil.isValidParam(pluginConfig, RABBITMQ_CONFIG.key())) { pluginConfig @@ -259,6 +286,15 @@ public RabbitmqConfig(Config config) { if (config.hasPath(USE_CORRELATION_ID.key())) { this.usesCorrelationId = config.getBoolean(USE_CORRELATION_ID.key()); } + if (config.hasPath(DURABLE.key())) { + this.durable = config.getBoolean(DURABLE.key()); + } + if (config.hasPath(EXCLUSIVE.key())) { + this.exclusive = config.getBoolean(EXCLUSIVE.key()); + } + if (config.hasPath(AUTO_DELETE.key())) { + this.autoDelete = config.getBoolean(AUTO_DELETE.key()); + } parseSinkOptionProperties(config); } diff --git a/seatunnel-connectors-v2/connector-typesense/pom.xml b/seatunnel-connectors-v2/connector-typesense/pom.xml new file mode 100644 index 00000000000..57a8682cb44 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/pom.xml @@ -0,0 +1,90 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-typesense + SeaTunnel : Connectors V2 : Typesense + + + 0.8.1 + + + + + com.fasterxml.jackson.core + jackson-databind + 2.14.1 + + + + org.typesense + typesense-java + 0.8.1 + + + org.apache.seatunnel + connector-common + ${project.version} + compile + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + shade + + package + + + + okhttp3 + shaded.okhttp3 + + + okio + shaded.okio + + + false + + + + + + + + diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java new file mode 100644 index 00000000000..fd73c247497 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java @@ -0,0 +1,214 @@ +/* + * 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.typesense.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.ConfigUtil; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PreviewResult; +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.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkNotNull; + +@Slf4j +public class TypesenseCatalog implements Catalog { + + private final String catalogName; + private final String defaultDatabase; + + private final ReadonlyConfig config; + private TypesenseClient typesenseClient; + + public TypesenseCatalog(String catalogName, String defaultDatabase, ReadonlyConfig config) { + this.catalogName = checkNotNull(catalogName, "catalogName cannot be null"); + this.defaultDatabase = defaultDatabase; + this.config = checkNotNull(config, "Typesense Config cannot be null"); + } + + @Override + public void open() throws CatalogException { + typesenseClient = TypesenseClient.createInstance(config); + } + + @Override + public void close() throws CatalogException { + // Nothing + } + + @Override + public String name() { + return catalogName; + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return defaultDatabase; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return typesenseClient.collectionExists(databaseName); + } + + @Override + public List listDatabases() throws CatalogException { + return typesenseClient.collectionList(); + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(catalogName, databaseName); + } + return Arrays.asList(databaseName); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + checkNotNull(tablePath); + return databaseExists(tablePath.getTableName()); + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + checkNotNull(tablePath, "tablePath cannot be null"); + TableSchema.Builder builder = TableSchema.builder(); + Map> fieldTypeMapping = + typesenseClient.getFieldTypeMapping(tablePath.getTableName()); + buildColumnsWithErrorCheck( + tablePath, + builder, + fieldTypeMapping.entrySet().iterator(), + nameAndType -> { + return PhysicalColumn.of( + nameAndType.getKey(), + TypesenseTypeConverter.INSTANCE + .convert(nameAndType.getValue()) + .getDataType(), + (Long) null, + true, + null, + null); + }); + + return CatalogTable.of( + TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + builder.build(), + buildTableOptions(tablePath), + Collections.emptyList(), + ""); + } + + private Map buildTableOptions(TablePath tablePath) { + Map options = new HashMap<>(); + options.put("connector", "typesense"); + options.put("config", ConfigUtil.convertToJsonString(tablePath)); + return options; + } + + @Override + public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + typesenseClient.createCollection(tablePath.getTableName()); + } + + @Override + public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath); + if (!tableExists(tablePath) && !ignoreIfNotExists) { + throw new TableNotExistException(catalogName, tablePath); + } + try { + typesenseClient.dropCollection(tablePath.getTableName()); + } catch (Exception ex) { + throw new CatalogException( + String.format( + "Failed to drop table %s in catalog %s", + tablePath.getTableName(), catalogName), + ex); + } + } + + @Override + public void createDatabase(TablePath tablePath, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + createTable(tablePath, null, ignoreIfExists); + } + + @Override + public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + dropTable(tablePath, ignoreIfNotExists); + } + + @Override + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) { + typesenseClient.truncateCollectionData(tablePath.getTableName()); + } + + @Override + public boolean isExistsData(TablePath tablePath) { + return typesenseClient.collectionDocNum(tablePath.getTableName()) > 0; + } + + @Override + public PreviewResult previewAction( + ActionType actionType, TablePath tablePath, Optional catalogTable) { + if (actionType == ActionType.CREATE_TABLE) { + return new InfoPreviewResult("create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_TABLE) { + return new InfoPreviewResult("delete collection " + tablePath.getTableName()); + } else if (actionType == ActionType.TRUNCATE_TABLE) { + return new InfoPreviewResult( + "delete and create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.CREATE_DATABASE) { + return new InfoPreviewResult("create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_DATABASE) { + return new InfoPreviewResult("delete collection " + tablePath.getTableName()); + } else { + throw new UnsupportedOperationException("Unsupported action type: " + actionType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java new file mode 100644 index 00000000000..0b619ad3052 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.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 com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class TypesenseCatalogFactory implements CatalogFactory { + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + return new TypesenseCatalog(catalogName, "", options); + } + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().build(); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java new file mode 100644 index 00000000000..c4cb862e29b --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeConverter; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.converter.TypeConverter; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; + +import com.google.auto.service.AutoService; + +import java.util.Map; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT32; +import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT64; + +@AutoService(TypeConverter.class) +public class TypesenseTypeConverter implements BasicTypeConverter> { + public static final TypesenseTypeConverter INSTANCE = new TypesenseTypeConverter(); + + @Override + public String identifier() { + return "Typesense"; + } + + @Override + public Column convert(BasicTypeDefine typeDefine) { + PhysicalColumn.PhysicalColumnBuilder builder = + PhysicalColumn.builder() + .name(typeDefine.getName()) + .sourceType(typeDefine.getColumnType()) + .nullable(typeDefine.isNullable()) + .defaultValue(typeDefine.getDefaultValue()) + .comment(typeDefine.getComment()); + String type = typeDefine.getDataType().toLowerCase(); + switch (type) { + case INT32: + builder.dataType(BasicType.INT_TYPE); + break; + case INT64: + builder.dataType(BasicType.LONG_TYPE); + break; + case TypesenseType.FLOAT: + builder.dataType(BasicType.FLOAT_TYPE); + break; + case TypesenseType.BOOL: + builder.dataType(BasicType.BOOLEAN_TYPE); + break; + case TypesenseType.OBJET: + Map> typeInfo = + (Map) typeDefine.getNativeType().getOptions(); + SeaTunnelRowType object = + new SeaTunnelRowType( + typeInfo.keySet().toArray(new String[0]), + typeInfo.values().stream() + .map(this::convert) + .map(Column::getDataType) + .toArray(SeaTunnelDataType[]::new)); + builder.dataType(object); + break; + case TypesenseType.STRING: + case TypesenseType.IMAGE: + default: + builder.dataType(BasicType.STRING_TYPE); + break; + } + return builder.build(); + } + + @Override + public BasicTypeDefine reconvert(Column column) { + throw new UnsupportedOperationException("Unsupported operation"); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java new file mode 100644 index 00000000000..01b848492fb --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java @@ -0,0 +1,286 @@ +/* + * 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.typesense.client; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.typesense.util.URLParamsConverter; + +import org.apache.commons.lang3.StringUtils; + +import org.typesense.api.Client; +import org.typesense.api.Collections; +import org.typesense.api.Configuration; +import org.typesense.api.FieldTypes; +import org.typesense.model.CollectionResponse; +import org.typesense.model.CollectionSchema; +import org.typesense.model.DeleteDocumentsParameters; +import org.typesense.model.Field; +import org.typesense.model.ImportDocumentsParameters; +import org.typesense.model.SearchParameters; +import org.typesense.model.SearchResult; +import org.typesense.resources.Node; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.extern.slf4j.Slf4j; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.QUERY_BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.CREATE_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DELETE_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DROP_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.FIELD_TYPE_MAPPING_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_EXISTS_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_LIST_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_NUM_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.TRUNCATE_COLLECTION_ERROR; + +@Slf4j +public class TypesenseClient { + private final Client tsClient; + + TypesenseClient(Client tsClient) { + this.tsClient = tsClient; + } + + public static TypesenseClient createInstance(ReadonlyConfig config) { + List hosts = config.get(TypesenseConnectionConfig.HOSTS); + String protocol = config.get(TypesenseConnectionConfig.protocol); + String apiKey = config.get(TypesenseConnectionConfig.APIKEY); + return createInstance(hosts, apiKey, protocol); + } + + public static TypesenseClient createInstance( + List hosts, String apiKey, String protocol) { + List nodes = new ArrayList<>(); + + hosts.stream() + .map(host -> host.split(":")) + .forEach( + split -> + nodes.add( + new Node( + protocol, + split[0], + StringUtils.isBlank(split[1]) + ? "8018" + : split[1]))); + + Configuration configuration = new Configuration(nodes, Duration.ofSeconds(5), apiKey); + Client client = new Client(configuration); + return new TypesenseClient(client); + } + + public void insert(String collection, List documentList) { + + ImportDocumentsParameters queryParameters = new ImportDocumentsParameters(); + queryParameters.action("upsert"); + String text = ""; + for (String s : documentList) { + text = text + s + "\n"; + } + try { + tsClient.collections(collection).documents().import_(text, queryParameters); + } catch (Exception e) { + log.error(INSERT_DOC_ERROR.getDescription()); + throw new TypesenseConnectorException( + INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); + } + } + + public SearchResult search(String collection, String query, int offset) throws Exception { + return search(collection, query, offset, QUERY_BATCH_SIZE.defaultValue()); + } + + public SearchResult search(String collection, String query, int offset, int pageSize) + throws Exception { + SearchParameters searchParameters; + if (StringUtils.isNotBlank(query)) { + String jsonQuery = URLParamsConverter.convertParamsToJson(query); + ObjectMapper objectMapper = new ObjectMapper(); + searchParameters = objectMapper.readValue(jsonQuery, SearchParameters.class); + } else { + searchParameters = new SearchParameters().q("*"); + } + log.debug("Typesense query param:{}", searchParameters); + searchParameters.offset(offset); + searchParameters.perPage(pageSize); + SearchResult searchResult = + tsClient.collections(collection).documents().search(searchParameters); + return searchResult; + } + + public boolean collectionExists(String collection) { + try { + Collections collections = tsClient.collections(); + CollectionResponse[] collectionResponses = collections.retrieve(); + for (CollectionResponse collectionRespons : collectionResponses) { + String collectionName = collectionRespons.getName(); + if (collection.equals(collectionName)) { + return true; + } + } + } catch (Exception e) { + log.error(QUERY_COLLECTION_EXISTS_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_EXISTS_ERROR, QUERY_COLLECTION_EXISTS_ERROR.getDescription()); + } + return false; + } + + public List collectionList() { + try { + Collections collections = tsClient.collections(); + CollectionResponse[] collectionResponses = collections.retrieve(); + List list = new ArrayList<>(); + for (CollectionResponse collectionRespons : collectionResponses) { + String collectionName = collectionRespons.getName(); + list.add(collectionName); + } + return list; + } catch (Exception e) { + log.error(QUERY_COLLECTION_LIST_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_LIST_ERROR, QUERY_COLLECTION_LIST_ERROR.getDescription()); + } + } + + public Map getField(String collection) { + if (collectionExists(collection)) { + Map fieldMap = new HashMap<>(); + try { + CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); + List fields = collectionResponse.getFields(); + for (Field field : fields) { + String fieldName = field.getName(); + String type = field.getType(); + fieldMap.put(fieldName, type); + } + } catch (Exception e) { + log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); + throw new TypesenseConnectorException( + FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); + } + return fieldMap; + } else { + return null; + } + } + + public Map> getFieldTypeMapping(String collection) { + Map> allTypesenseSearchFieldTypeInfoMap = + new HashMap<>(); + try { + CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); + List fields = collectionResponse.getFields(); + for (Field field : fields) { + String fieldName = field.getName(); + String type = field.getType(); + BasicTypeDefine.BasicTypeDefineBuilder typeDefine = + BasicTypeDefine.builder() + .name(fieldName) + .columnType(type) + .dataType(type) + .nativeType(new TypesenseType(type, new HashMap<>())); + allTypesenseSearchFieldTypeInfoMap.put(fieldName, typeDefine.build()); + } + } catch (Exception e) { + log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); + throw new TypesenseConnectorException( + FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); + } + return allTypesenseSearchFieldTypeInfoMap; + } + + public boolean createCollection(String collection) { + if (collectionExists(collection)) { + return true; + } + List fields = new ArrayList<>(); + fields.add(new Field().name(".*").type(FieldTypes.AUTO)); + return createCollection(collection, fields); + } + + public boolean createCollection(String collection, List fields) { + CollectionSchema collectionSchema = new CollectionSchema(); + collectionSchema.name(collection).fields(fields).enableNestedFields(true); + try { + tsClient.collections().create(collectionSchema); + return true; + } catch (Exception e) { + log.error(CREATE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + CREATE_COLLECTION_ERROR, CREATE_COLLECTION_ERROR.getDescription()); + } + } + + public boolean dropCollection(String collection) { + try { + tsClient.collections(collection).delete(); + return true; + } catch (Exception e) { + log.error(DROP_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + DROP_COLLECTION_ERROR, DROP_COLLECTION_ERROR.getDescription()); + } + } + + public boolean truncateCollectionData(String collection) { + DeleteDocumentsParameters deleteDocumentsParameters = new DeleteDocumentsParameters(); + deleteDocumentsParameters.filterBy("id:!=1||id:=1"); + try { + tsClient.collections(collection).documents().delete(deleteDocumentsParameters); + } catch (Exception e) { + log.error(TRUNCATE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + TRUNCATE_COLLECTION_ERROR, TRUNCATE_COLLECTION_ERROR.getDescription()); + } + return true; + } + + public boolean deleteCollectionData(String collection, String id) { + try { + tsClient.collections(collection).documents(id).delete(); + } catch (Exception e) { + log.error(DELETE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + DELETE_COLLECTION_ERROR, DELETE_COLLECTION_ERROR.getDescription()); + } + return true; + } + + public long collectionDocNum(String collection) { + SearchParameters q = new SearchParameters().q("*"); + try { + SearchResult searchResult = tsClient.collections(collection).documents().search(q); + return searchResult.getFound(); + } catch (Exception e) { + log.error(QUERY_COLLECTION_NUM_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_NUM_ERROR, QUERY_COLLECTION_NUM_ERROR.getDescription()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java new file mode 100644 index 00000000000..2ac05a6d312 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.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.typesense.client; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.util.Map; + +@Getter +@AllArgsConstructor +public class TypesenseType { + + public static final String STRING = "string"; + public static final String INT32 = "int32"; + public static final String INT64 = "int64"; + public static final String FLOAT = "float"; + public static final String BOOL = "bool"; + public static final String IMAGE = "image"; + public static final String OBJET = "object"; + private String type; + private Map options; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java new file mode 100644 index 00000000000..9912c6aa2e1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java @@ -0,0 +1,78 @@ +/* + * 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.typesense.config; + +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 java.util.Arrays; +import java.util.List; + +import static org.apache.seatunnel.api.sink.DataSaveMode.APPEND_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.DROP_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.ERROR_WHEN_DATA_EXISTS; + +public class SinkConfig { + + public static final Option COLLECTION = + Options.key("collection") + .stringType() + .noDefaultValue() + .withDescription("Typesense collection name"); + + public static final Option> PRIMARY_KEYS = + Options.key("primary_keys") + .listType(String.class) + .noDefaultValue() + .withDescription("Primary key fields used to generate the document `id`"); + + public static final Option KEY_DELIMITER = + Options.key("key_delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys (\"_\" by default), e.g., \"$\" would result in document `id` \"KEY1$KEY2$KEY3\"."); + + public static final Option MAX_BATCH_SIZE = + Options.key("max_batch_size") + .intType() + .defaultValue(10) + .withDescription("batch bulk doc max size"); + + public static final Option MAX_RETRY_COUNT = + Options.key("max_retry_count") + .intType() + .defaultValue(3) + .withDescription("one bulk request max try count"); + + public static final Option SCHEMA_SAVE_MODE = + Options.key("schema_save_mode") + .enumType(SchemaSaveMode.class) + .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) + .withDescription("schema_save_mode"); + + public static final Option DATA_SAVE_MODE = + Options.key("data_save_mode") + .singleChoice( + DataSaveMode.class, + Arrays.asList(DROP_DATA, APPEND_DATA, ERROR_WHEN_DATA_EXISTS)) + .defaultValue(APPEND_DATA) + .withDescription("data_save_mode"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java new file mode 100644 index 00000000000..a1642331a6c --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +public class SourceConfig { + + public static final Option COLLECTION = + Options.key("collection") + .stringType() + .noDefaultValue() + .withDescription("Typesense collection name"); + + public static final Option QUERY = + Options.key("query") + .stringType() + .noDefaultValue() + .withDescription("Typesense query param"); + + public static final Option QUERY_BATCH_SIZE = + Options.key("batch_size") + .intType() + .defaultValue(100) + .withDescription("Typesense query batch size"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java new file mode 100644 index 00000000000..2f36dc044c5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import java.util.List; + +public class TypesenseConnectionConfig { + + public static final Option> HOSTS = + Options.key("hosts") + .listType() + .noDefaultValue() + .withDescription( + "Typesense cluster http address, the format is host:port, allowing multiple hosts to be specified. Such as [\"host1:8018\", \"host2:8018\"]"); + + public static final Option APIKEY = + Options.key("api_key") + .stringType() + .noDefaultValue() + .withDescription("Typesense api key"); + + public static final Option protocol = + Options.key("protocol") + .stringType() + .defaultValue("http") + .withDescription("Default is http , for Typesense Cloud use https"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java new file mode 100644 index 00000000000..9fd4892979f --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java @@ -0,0 +1,40 @@ +/* + * 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.typesense.dto; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; + +import lombok.Data; + +@Data +public class CollectionInfo { + + private String collection; + private String type; + private String[] primaryKeys; + private String keyDelimiter; + + public CollectionInfo(String collection, ReadonlyConfig config) { + this.collection = collection; + if (config.getOptional(SinkConfig.PRIMARY_KEYS).isPresent()) { + primaryKeys = config.get(SinkConfig.PRIMARY_KEYS).toArray(new String[0]); + } + keyDelimiter = config.get(SinkConfig.KEY_DELIMITER); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java new file mode 100644 index 00000000000..9e1b55f8cbd --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.dto; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; + +@Data +@AllArgsConstructor +public class SourceCollectionInfo implements Serializable { + private String collection; + private String query; + private long found; + private int offset; + private int queryBatchSize; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java new file mode 100644 index 00000000000..8edcf608d55 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum TypesenseConnectorErrorCode implements SeaTunnelErrorCode { + QUERY_PARAM_ERROR("TYPESENSE-01", "Query parameter error"), + QUERY_COLLECTION_EXISTS_ERROR("TYPESENSE-02", "Whether the collection stores query exceptions"), + QUERY_COLLECTION_LIST_ERROR("TYPESENSE-03", "Collection list acquisition exception"), + FIELD_TYPE_MAPPING_ERROR("TYPESENSE-04", "Failed to obtain the field"), + CREATE_COLLECTION_ERROR("TYPESENSE-05", "Create collection failed"), + DROP_COLLECTION_ERROR("TYPESENSE-06", "Drop collection failed"), + TRUNCATE_COLLECTION_ERROR("TYPESENSE-07", "Truncate collection failed"), + QUERY_COLLECTION_NUM_ERROR("TYPESENSE-08", "Query collection doc number failed"), + INSERT_DOC_ERROR("TYPESENSE-09", "Insert documents failed"), + DELETE_COLLECTION_ERROR("TYPESENSE-10", "Truncate collection failed"); + private final String code; + private final String description; + + TypesenseConnectorErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java new file mode 100644 index 00000000000..3c1cb1cc167 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.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.typesense.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class TypesenseConnectorException extends SeaTunnelRuntimeException { + public TypesenseConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public TypesenseConnectorException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java new file mode 100644 index 00000000000..5dc56f0bb77 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import lombok.AllArgsConstructor; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +@AllArgsConstructor +public class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + @Override + public String apply(SeaTunnelRow row) { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + String value = fieldFormatters[i].format(row); + builder.append(value); + } + return builder.toString(); + } + + public static Function createKeyExtractor( + SeaTunnelRowType rowType, String[] primaryKeys, String keyDelimiter) { + if (primaryKeys == null) { + return row -> null; + } + + List fieldFormatters = new ArrayList<>(primaryKeys.length); + for (String fieldName : primaryKeys) { + int fieldIndex = rowType.indexOf(fieldName); + SeaTunnelDataType fieldType = rowType.getFieldType(fieldIndex); + FieldFormatter fieldFormatter = createFieldFormatter(fieldIndex, fieldType); + fieldFormatters.add(fieldFormatter); + } + return new KeyExtractor(fieldFormatters.toArray(new FieldFormatter[0]), keyDelimiter); + } + + private static FieldFormatter createFieldFormatter( + int fieldIndex, SeaTunnelDataType fieldType) { + return row -> { + switch (fieldType.getSqlType()) { + case ROW: + case ARRAY: + case MAP: + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported type: " + fieldType); + case DATE: + LocalDate localDate = (LocalDate) row.getField(fieldIndex); + return localDate.toString(); + case TIME: + LocalTime localTime = (LocalTime) row.getField(fieldIndex); + return localTime.toString(); + case TIMESTAMP: + LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); + return localDateTime.toString(); + default: + return row.getField(fieldIndex).toString(); + } + }; + } + + private interface FieldFormatter extends Serializable { + String format(SeaTunnelRow row); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java new file mode 100644 index 00000000000..94da50cf09b --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface SeaTunnelRowSerializer { + String serializeRow(SeaTunnelRow row); + + String serializeRowForDelete(SeaTunnelRow row); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java new file mode 100644 index 00000000000..07faca4cc92 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java @@ -0,0 +1,119 @@ +/* + * 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.typesense.serialize.sink; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.KeyExtractor; + +import org.apache.commons.lang3.StringUtils; + +import java.time.temporal.Temporal; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class TypesenseRowSerializer implements SeaTunnelRowSerializer { + + private final SeaTunnelRowType seaTunnelRowType; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + private final Function keyExtractor; + + public TypesenseRowSerializer( + CollectionInfo collectionInfo, SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + this.keyExtractor = + KeyExtractor.createKeyExtractor( + seaTunnelRowType, + collectionInfo.getPrimaryKeys(), + collectionInfo.getKeyDelimiter()); + } + + @Override + public String serializeRow(SeaTunnelRow row) { + String key = keyExtractor.apply(row); + Map document = toDocumentMap(row, seaTunnelRowType); + if (StringUtils.isNotBlank(key)) { + document.put("id", key); + } + String documentStr; + try { + documentStr = objectMapper.writeValueAsString(document); + } catch (JsonProcessingException e) { + throw CommonError.jsonOperationError("Typesense", "document:" + document.toString(), e); + } + return documentStr; + } + + @Override + public String serializeRowForDelete(SeaTunnelRow row) { + String key = keyExtractor.apply(row); + Map document = toDocumentMap(row, seaTunnelRowType); + String id = document.get("id").toString(); + if (StringUtils.isNotBlank(key)) { + id = key; + } + return id; + } + + private Map toDocumentMap(SeaTunnelRow row, SeaTunnelRowType rowType) { + String[] fieldNames = rowType.getFieldNames(); + Map doc = new HashMap<>(fieldNames.length); + Object[] fields = row.getFields(); + for (int i = 0; i < fieldNames.length; i++) { + Object value = fields[i]; + if (value == null) { + } else if (value instanceof SeaTunnelRow) { + doc.put( + fieldNames[i], + toDocumentMap( + (SeaTunnelRow) value, (SeaTunnelRowType) rowType.getFieldType(i))); + } else { + doc.put(fieldNames[i], convertValue(value)); + } + } + return doc; + } + + private Object convertValue(Object value) { + if (value instanceof Temporal) { + // jackson not support jdk8 new time api + return value.toString(); + } else if (value instanceof Map) { + for (Map.Entry entry : ((Map) value).entrySet()) { + ((Map) value).put(entry.getKey(), convertValue(entry.getValue())); + } + return value; + } else if (value instanceof List) { + for (int i = 0; i < ((List) value).size(); i++) { + ((List) value).set(i, convertValue(((List) value).get(i))); + } + return value; + } else { + return value; + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java new file mode 100644 index 00000000000..d13901b7081 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java @@ -0,0 +1,24 @@ +/* + * 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.typesense.serialize.sink.collection; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface CollectionSerializer { + String serialize(SeaTunnelRow row); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java new file mode 100644 index 00000000000..a2b0da248e1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.collection; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public class FixedValueCollectionSerializer implements CollectionSerializer { + + private final String index; + + public FixedValueCollectionSerializer(String index) { + this.index = index; + } + + @Override + public String serialize(SeaTunnelRow row) { + return index; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..762506d4980 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java @@ -0,0 +1,253 @@ +/* + * 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.typesense.serialize.source; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.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.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.seatunnel.api.table.type.BasicType.BOOLEAN_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.BYTE_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.DOUBLE_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; +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.SHORT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.VOID_TYPE; + +public class DefaultSeaTunnelRowDeserializer implements SeaTunnelRowDeserializer { + + private final SeaTunnelRowType rowTypeInfo; + + private final ObjectMapper mapper = new ObjectMapper(); + + private final String nullDefault = "null"; + + private final Map dateTimeFormatterMap = + new HashMap() { + { + put("yyyy-MM-dd HH".length(), DateTimeFormatter.ofPattern("yyyy-MM-dd HH")); + put( + "yyyy-MM-dd HH:mm".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm")); + put( + "yyyyMMdd HH:mm:ss".length(), + DateTimeFormatter.ofPattern("yyyyMMdd HH:mm:ss")); + put( + "yyyy-MM-dd HH:mm:ss".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")); + put( + "yyyy-MM-dd HH:mm:ss.S".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S")); + put( + "yyyy-MM-dd HH:mm:ss.SS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SS")); + put( + "yyyy-MM-dd HH:mm:ss.SSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSSSSSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS")); + } + }; + + public DefaultSeaTunnelRowDeserializer(SeaTunnelRowType rowTypeInfo) { + this.rowTypeInfo = rowTypeInfo; + } + + @Override + public SeaTunnelRow deserialize(TypesenseRecord rowRecord) { + return convert(rowRecord); + } + + SeaTunnelRow convert(TypesenseRecord rowRecord) { + Object[] seaTunnelFields = new Object[rowTypeInfo.getTotalFields()]; + String fieldName = null; + Object value = null; + SeaTunnelDataType seaTunnelDataType = null; + Map doc = rowRecord.getDoc(); + try { + for (int i = 0; i < rowTypeInfo.getTotalFields(); i++) { + fieldName = rowTypeInfo.getFieldName(i); + value = doc.get(fieldName); // 字段值 + if (value != null) { + seaTunnelDataType = + rowTypeInfo.getFieldType(i); // seaTunnelDataType 为SeaTunnel类型 + seaTunnelFields[i] = convertValue(seaTunnelDataType, value); + } + } + } catch (Exception ex) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + String.format( + "error fieldName=%s,fieldValue=%s,seaTunnelDataType=%s,rowRecord=%s", + fieldName, value, seaTunnelDataType, JsonUtils.toJsonString(rowRecord)), + ex); + } + return new SeaTunnelRow(seaTunnelFields); + } + + Object convertValue(SeaTunnelDataType fieldType, Object fieldValue) + throws JsonProcessingException { + if (STRING_TYPE.equals(fieldType)) { + return fieldValue.toString(); + } else { + if (nullDefault.equals(fieldValue.toString())) { + return null; + } + if (BOOLEAN_TYPE.equals(fieldType)) { + return Boolean.parseBoolean(fieldValue.toString()); + } else if (BYTE_TYPE.equals(fieldType)) { + return Byte.valueOf(fieldValue.toString()); + } else if (SHORT_TYPE.equals(fieldType)) { + return Short.parseShort(fieldValue.toString()); + } else if (INT_TYPE.equals(fieldType)) { + return Integer.parseInt(fieldValue.toString()); + } else if (LONG_TYPE.equals(fieldType)) { + return Long.parseLong(fieldValue.toString()); + } else if (FLOAT_TYPE.equals(fieldType)) { + return Float.parseFloat(fieldValue.toString()); + } else if (DOUBLE_TYPE.equals(fieldType)) { + return Double.parseDouble(fieldValue.toString()); + } else if (LocalTimeType.LOCAL_DATE_TYPE.equals(fieldType)) { + LocalDateTime localDateTime = parseDate(fieldValue.toString()); + return localDateTime.toLocalDate(); + } else if (LocalTimeType.LOCAL_TIME_TYPE.equals(fieldType)) { + LocalDateTime localDateTime = parseDate(fieldValue.toString()); + return localDateTime.toLocalTime(); + } else if (LocalTimeType.LOCAL_DATE_TIME_TYPE.equals(fieldType)) { + return parseDate(fieldValue.toString()); + } else if (fieldType instanceof DecimalType) { + return new BigDecimal(fieldValue.toString()); + } else if (fieldType instanceof ArrayType) { + ArrayType arrayType = (ArrayType) fieldType; + SeaTunnelDataType elementType = arrayType.getElementType(); + List stringList = (List) fieldValue; + Object arr = Array.newInstance(elementType.getTypeClass(), stringList.size()); + for (int i = 0; i < stringList.size(); i++) { + Object convertValue = convertValue(elementType, stringList.get(i)); + Array.set(arr, i, convertValue); + } + return arr; + } else if (fieldType instanceof MapType) { + MapType mapType = (MapType) fieldType; + SeaTunnelDataType keyType = mapType.getKeyType(); + + SeaTunnelDataType valueType = mapType.getValueType(); + Map stringMap = + mapper.readValue( + fieldValue.toString(), + new TypeReference>() {}); + Map convertMap = new HashMap(); + for (Map.Entry entry : stringMap.entrySet()) { + Object convertKey = convertValue(keyType, entry.getKey()); + Object convertValue = convertValue(valueType, entry.getValue()); + convertMap.put(convertKey, convertValue); + } + return convertMap; + } else if (fieldType instanceof SeaTunnelRowType) { + SeaTunnelRowType rowType = (SeaTunnelRowType) fieldType; + Map collect = (Map) fieldValue; + Object[] seaTunnelFields = new Object[rowType.getTotalFields()]; + for (int i = 0; i < rowType.getTotalFields(); i++) { + String fieldName = rowType.getFieldName(i); + SeaTunnelDataType fieldDataType = rowType.getFieldType(i); + Object value = collect.get(fieldName); + if (value != null) { + seaTunnelFields[i] = convertValue(fieldDataType, value); + } + } + return new SeaTunnelRow(seaTunnelFields); + } else if (fieldType instanceof PrimitiveByteArrayType) { + return Base64.getDecoder().decode(fieldValue.toString()); + } else if (VOID_TYPE.equals(fieldType) || fieldType == null) { + return null; + } else { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + fieldType); + } + } + } + + private LocalDateTime parseDate(String fieldValue) { + // handle strings of timestamp type + try { + long ts = Long.parseLong(fieldValue); + return LocalDateTime.ofInstant(Instant.ofEpochMilli(ts), ZoneId.systemDefault()); + } catch (NumberFormatException e) { + // no op + } + String formatDate = fieldValue.replace("T", " ").replace("Z", ""); + if (fieldValue.length() == "yyyyMMdd".length() + || fieldValue.length() == "yyyy-MM-dd".length()) { + formatDate = fieldValue + " 00:00:00"; + } + DateTimeFormatter dateTimeFormatter = dateTimeFormatterMap.get(formatDate.length()); + if (dateTimeFormatter == null) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, "unsupported date format"); + } + return LocalDateTime.parse(formatDate, dateTimeFormatter); + } + + Object recursiveGet(Map collect, String keyWithRecursive) { + Object value = null; + boolean isFirst = true; + for (String key : keyWithRecursive.split("\\.")) { + if (isFirst) { + value = collect.get(key); + isFirst = false; + } else if (value instanceof ObjectNode) { + value = ((ObjectNode) value).get(key); + } + } + return value; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..a5864e596bc --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java @@ -0,0 +1,25 @@ +/* + * 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.typesense.serialize.source; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface SeaTunnelRowDeserializer { + + SeaTunnelRow deserialize(TypesenseRecord rowRecord); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java new file mode 100644 index 00000000000..154b94ddc65 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.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.typesense.serialize.source; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +import java.util.Map; + +@Getter +@ToString +@AllArgsConstructor +public class TypesenseRecord { + private Map doc; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java new file mode 100644 index 00000000000..e52638f83ef --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java @@ -0,0 +1,96 @@ +/* + * 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.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SaveModeHandler; +import org.apache.seatunnel.api.sink.SchemaSaveMode; +import org.apache.seatunnel.api.sink.SeaTunnelSink; +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.factory.FactoryUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; + +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_RETRY_COUNT; + +public class TypesenseSink + implements SeaTunnelSink< + SeaTunnelRow, + TypesenseSinkState, + TypesenseCommitInfo, + TypesenseAggregatedCommitInfo>, + SupportMultiTableSink, + SupportSaveMode { + + private ReadonlyConfig config; + private CatalogTable catalogTable; + private final int maxBatchSize; + private final int maxRetryCount; + + public TypesenseSink(ReadonlyConfig config, CatalogTable catalogTable) { + this.config = config; + this.catalogTable = catalogTable; + maxBatchSize = config.get(MAX_BATCH_SIZE); + maxRetryCount = config.get(MAX_RETRY_COUNT); + } + + @Override + public String getPluginName() { + return "Typesense"; + } + + @Override + public TypesenseSinkWriter createWriter(SinkWriter.Context context) { + return new TypesenseSinkWriter(context, catalogTable, config, maxBatchSize, maxRetryCount); + } + + @Override + public Optional getSaveModeHandler() { + CatalogFactory catalogFactory = + FactoryUtil.discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + getPluginName()); + if (catalogFactory == null) { + return Optional.empty(); + } + Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); + SchemaSaveMode schemaSaveMode = config.get(SinkConfig.SCHEMA_SAVE_MODE); + DataSaveMode dataSaveMode = config.get(SinkConfig.DATA_SAVE_MODE); + + TablePath tablePath = TablePath.of("", catalogTable.getTableId().getTableName()); + catalog.open(); + return Optional.of( + new DefaultSaveModeHandler( + schemaSaveMode, dataSaveMode, catalog, tablePath, null, null)); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java new file mode 100644 index 00000000000..4c937609406 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.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.typesense.config.SinkConfig; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.COLLECTION; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.KEY_DELIMITER; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.PRIMARY_KEYS; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; + +@AutoService(Factory.class) +public class TypesenseSinkFactory implements TableSinkFactory { + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + HOSTS, + COLLECTION, + APIKEY, + SinkConfig.SCHEMA_SAVE_MODE, + SinkConfig.DATA_SAVE_MODE) + .optional(PRIMARY_KEYS, KEY_DELIMITER) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + ReadonlyConfig readonlyConfig = context.getOptions(); + String original = readonlyConfig.get(COLLECTION); + CatalogTable newTable = + CatalogTable.of( + TableIdentifier.of( + context.getCatalogTable().getCatalogName(), + context.getCatalogTable().getTablePath().getDatabaseName(), + original), + context.getCatalogTable()); + return () -> new TypesenseSink(readonlyConfig, newTable); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java new file mode 100644 index 00000000000..9fdaf68a242 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java @@ -0,0 +1,139 @@ +/* + * 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.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.utils.RetryUtils; +import org.apache.seatunnel.common.utils.RetryUtils.RetryMaterial; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.SeaTunnelRowSerializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; + +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static org.apache.seatunnel.api.table.type.RowKind.INSERT; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; + +@Slf4j +public class TypesenseSinkWriter + implements SinkWriter, + SupportMultiTableSinkWriter { + + private final Context context; + private final int maxBatchSize; + private final SeaTunnelRowSerializer seaTunnelRowSerializer; + + private final List requestEsList; + + private final String collection; + private TypesenseClient typesenseClient; + private RetryMaterial retryMaterial; + private static final long DEFAULT_SLEEP_TIME_MS = 200L; + + public TypesenseSinkWriter( + Context context, + CatalogTable catalogTable, + ReadonlyConfig config, + int maxBatchSize, + int maxRetryCount) { + this.context = context; + this.maxBatchSize = maxBatchSize; + + collection = catalogTable.getTableId().getTableName(); + CollectionInfo collectionInfo = + new CollectionInfo(catalogTable.getTableId().getTableName(), config); + typesenseClient = TypesenseClient.createInstance(config); + this.seaTunnelRowSerializer = + new TypesenseRowSerializer(collectionInfo, catalogTable.getSeaTunnelRowType()); + + this.requestEsList = new ArrayList<>(maxBatchSize); + this.retryMaterial = + new RetryMaterial(maxRetryCount, true, exception -> true, DEFAULT_SLEEP_TIME_MS); + } + + @Override + public void write(SeaTunnelRow element) { + if (RowKind.UPDATE_BEFORE.equals(element.getRowKind())) { + return; + } + + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + String indexRequestRow = seaTunnelRowSerializer.serializeRow(element); + requestEsList.add(indexRequestRow); + if (requestEsList.size() >= maxBatchSize) { + insert(collection, requestEsList); + } + break; + case UPDATE_BEFORE: + case DELETE: + String id = seaTunnelRowSerializer.serializeRowForDelete(element); + typesenseClient.deleteCollectionData(collection, id); + break; + default: + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported write row kind: " + element.getRowKind()); + } + } + + @Override + public Optional prepareCommit() { + insert(this.collection, this.requestEsList); + return Optional.empty(); + } + + private void insert(String collection, List requestEsList) { + try { + RetryUtils.retryWithException( + () -> { + typesenseClient.insert(collection, requestEsList); + return null; + }, + retryMaterial); + requestEsList.clear(); + } catch (Exception e) { + log.error(INSERT_DOC_ERROR.getDescription()); + throw new TypesenseConnectorException( + INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); + } + } + + @Override + public void abortPrepare() {} + + @Override + public void close() { + insert(collection, requestEsList); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java new file mode 100644 index 00000000000..a3860a0495a --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java @@ -0,0 +1,87 @@ +/* + * 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.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Collections; +import java.util.List; + +@Slf4j +public class TypesenseSource + implements SeaTunnelSource, + SupportParallelism, + SupportColumnProjection { + + private final ReadonlyConfig config; + + private CatalogTable catalogTable; + + public TypesenseSource(ReadonlyConfig config) { + this.config = config; + if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { + catalogTable = CatalogTableUtil.buildWithConfig(config); + } + } + + @Override + public String getPluginName() { + return "Typesense"; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public List getProducedCatalogTables() { + return Collections.singletonList(catalogTable); + } + + @Override + public SourceReader createReader( + SourceReader.Context readerContext) throws Exception { + return new TypesenseSourceReader(readerContext, config, catalogTable.getSeaTunnelRowType()); + } + + @Override + public SourceSplitEnumerator createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) { + return new TypesenseSourceSplitEnumerator(enumeratorContext, config); + } + + @Override + public SourceSplitEnumerator restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + TypesenseSourceState checkpointState) { + return new TypesenseSourceSplitEnumerator(enumeratorContext, config); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java new file mode 100644 index 00000000000..254763ebbb1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java @@ -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. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.COLLECTION; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; + +@AutoService(Factory.class) +public class TypesenseSourceFactory implements TableSourceFactory { + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().required(HOSTS, APIKEY).optional(COLLECTION).build(); + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new TypesenseSource(context.getOptions()); + } + + @Override + public Class getSourceClass() { + return TypesenseSource.class; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java new file mode 100644 index 00000000000..16946519968 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.DefaultSeaTunnelRowDeserializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.SeaTunnelRowDeserializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.TypesenseRecord; + +import org.typesense.model.SearchResult; +import org.typesense.model.SearchResultHit; + +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +@Slf4j +public class TypesenseSourceReader implements SourceReader { + + SourceReader.Context context; + + private final ReadonlyConfig config; + + private final SeaTunnelRowDeserializer deserializer; + + private TypesenseClient typesenseClient; + + Deque splits = new LinkedList<>(); + + boolean noMoreSplit; + + private final long pollNextWaitTime = 1000L; + + public TypesenseSourceReader( + SourceReader.Context context, ReadonlyConfig config, SeaTunnelRowType rowTypeInfo) { + this.context = context; + this.config = config; + this.deserializer = new DefaultSeaTunnelRowDeserializer(rowTypeInfo); + } + + @Override + public void open() { + typesenseClient = TypesenseClient.createInstance(this.config); + } + + @Override + public void close() { + // Nothing , because typesense does not require + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + return new ArrayList<>(splits); + } + + @Override + public void addSplits(List splits) { + this.splits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + noMoreSplit = true; + } + + @Override + public void pollNext(Collector output) throws Exception { + synchronized (output.getCheckpointLock()) { + TypesenseSourceSplit split = splits.poll(); + if (split != null) { + SourceCollectionInfo sourceCollectionInfo = split.getSourceCollectionInfo(); + int pageSize = sourceCollectionInfo.getQueryBatchSize(); + while (true) { + SearchResult searchResult = + typesenseClient.search( + sourceCollectionInfo.getCollection(), + sourceCollectionInfo.getQuery(), + sourceCollectionInfo.getOffset(), + sourceCollectionInfo.getQueryBatchSize()); + Integer found = searchResult.getFound(); + List hits = searchResult.getHits(); + for (SearchResultHit hit : hits) { + Map document = hit.getDocument(); + SeaTunnelRow seaTunnelRow = + deserializer.deserialize(new TypesenseRecord(document)); + output.collect(seaTunnelRow); + } + if ((double) found / pageSize - 1 + > sourceCollectionInfo.getOffset() / pageSize) { + sourceCollectionInfo.setOffset(sourceCollectionInfo.getOffset() + pageSize); + } else { + break; + } + } + + } else if (noMoreSplit) { + log.info("Closed the bounded Typesense source"); + context.signalNoMoreElement(); + } else { + Thread.sleep(pollNextWaitTime); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java new file mode 100644 index 00000000000..39cf8530eec --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +@ToString +@AllArgsConstructor +public class TypesenseSourceSplit implements SourceSplit { + + private static final long serialVersionUID = -1L; + + private String splitId; + + @Getter private SourceCollectionInfo sourceCollectionInfo; + + @Override + public String splitId() { + return splitId; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java new file mode 100644 index 00000000000..adb149ae4f7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java @@ -0,0 +1,187 @@ +/* + * 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.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@Slf4j +public class TypesenseSourceSplitEnumerator + implements SourceSplitEnumerator { + + private final SourceSplitEnumerator.Context context; + + private final ReadonlyConfig config; + + private TypesenseClient typesenseClient; + + private final Object stateLock = new Object(); + + private Map> pendingSplit; + + private volatile boolean shouldEnumerate; + + public TypesenseSourceSplitEnumerator( + SourceSplitEnumerator.Context context, ReadonlyConfig config) { + this(context, null, config); + } + + public TypesenseSourceSplitEnumerator( + SourceSplitEnumerator.Context context, + TypesenseSourceState sourceState, + ReadonlyConfig config) { + this.context = context; + this.config = config; + this.pendingSplit = new HashMap<>(); + this.shouldEnumerate = sourceState == null; + if (sourceState != null) { + this.shouldEnumerate = sourceState.isShouldEnumerate(); + this.pendingSplit.putAll(sourceState.getPendingSplit()); + } + } + + @Override + public void open() { + // Nothing + } + + @Override + public void run() throws Exception { + Set readers = context.registeredReaders(); + if (shouldEnumerate) { + List newSplits = getTypesenseSplit(); + + synchronized (stateLock) { + addPendingSplit(newSplits); + shouldEnumerate = false; + } + + assignSplit(readers); + } + + log.debug( + "No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); + readers.forEach(context::signalNoMoreSplits); + } + + private void addPendingSplit(Collection splits) { + int readerCount = context.currentParallelism(); + for (TypesenseSourceSplit split : splits) { + int ownerReader = getSplitOwner(split.splitId(), readerCount); + log.info("Assigning {} to {} reader.", split, ownerReader); + pendingSplit.computeIfAbsent(ownerReader, r -> new ArrayList<>()).add(split); + } + } + + private void assignSplit(Collection readers) { + log.debug("Assign pendingSplits to readers {}", readers); + + for (int reader : readers) { + List assignmentForReader = pendingSplit.remove(reader); + if (assignmentForReader != null && !assignmentForReader.isEmpty()) { + log.info("Assign splits {} to reader {}", assignmentForReader, reader); + try { + context.assignSplit(reader, assignmentForReader); + } catch (Exception e) { + log.error( + "Failed to assign splits {} to reader {}", + assignmentForReader, + reader, + e); + pendingSplit.put(reader, assignmentForReader); + } + } + } + } + + private static int getSplitOwner(String tp, int numReaders) { + return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; + } + + private List getTypesenseSplit() { + List splits = new ArrayList<>(); + + String collection = config.get(SourceConfig.COLLECTION); + String query = config.get(SourceConfig.QUERY); + int queryBatchSize = config.get(SourceConfig.QUERY_BATCH_SIZE); + splits.add( + new TypesenseSourceSplit( + collection, + new SourceCollectionInfo(collection, query, 0, 0, queryBatchSize))); + return splits; + } + + @Override + public void close() throws IOException { + // Nothing + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + if (!splits.isEmpty()) { + addPendingSplit(splits); + assignSplit(Collections.singletonList(subtaskId)); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplit.size(); + } + + @Override + public void handleSplitRequest(int subtaskId) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported handleSplitRequest: " + subtaskId); + } + + @Override + public void registerReader(int subtaskId) { + log.debug("Register reader {} to IoTDBSourceSplitEnumerator.", subtaskId); + if (!pendingSplit.isEmpty()) { + assignSplit(Collections.singletonList(subtaskId)); + } + } + + @Override + public TypesenseSourceState snapshotState(long checkpointId) throws Exception { + synchronized (stateLock) { + return new TypesenseSourceState(shouldEnumerate, pendingSplit); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java new file mode 100644 index 00000000000..e29c7326102 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.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.typesense.source; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +@AllArgsConstructor +@Getter +public class TypesenseSourceState implements Serializable { + private boolean shouldEnumerate; + private Map> pendingSplit; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java new file mode 100644 index 00000000000..b0f814dd8fd --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java @@ -0,0 +1,22 @@ +/* + * 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.typesense.state; + +import java.io.Serializable; + +public class TypesenseAggregatedCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java new file mode 100644 index 00000000000..05e0ea83f72 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java @@ -0,0 +1,22 @@ +/* + * 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.typesense.state; + +import java.io.Serializable; + +public class TypesenseCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java new file mode 100644 index 00000000000..99f341d78cc --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java @@ -0,0 +1,22 @@ +/* + * 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.typesense.state; + +import java.io.Serializable; + +public class TypesenseSinkState implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java new file mode 100644 index 00000000000..cc6f26cd43f --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java @@ -0,0 +1,71 @@ +/* + * 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.typesense.util; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class URLParamsConverter { + + public static String convertParamsToJson(String paramsString) { + return Optional.ofNullable(paramsString) + .filter(s -> !s.isEmpty()) + .map(URLParamsConverter::parseParams) + .map( + paramsMap -> { + try { + return new ObjectMapper().writeValueAsString(paramsMap); + } catch (IOException e) { + throw new RuntimeException("Error converting params to JSON", e); + } + }) + .orElseThrow( + () -> + new IllegalArgumentException( + "Parameter string must not be null or empty.")); + } + + private static Map parseParams(String paramsString) { + return Arrays.stream( + Optional.ofNullable(paramsString) + .filter(s -> !s.isEmpty()) + .orElseThrow( + () -> + new IllegalArgumentException( + "Parameter string must not be null or empty.")) + .split("&")) + .map(part -> part.split("=", 2)) + .peek( + keyValue -> { + if (keyValue.length != 2) { + throw new TypesenseConnectorException( + TypesenseConnectorErrorCode.QUERY_PARAM_ERROR, + "Query parameter error: " + Arrays.toString(keyValue)); + } + }) + .collect(Collectors.toMap(keyValue -> keyValue[0], keyValue -> keyValue[1])); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java new file mode 100644 index 00000000000..bb604869ff1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serializer; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; + +public class TypesenseRowSerializerTest { + @Test + public void testSerializeUpsert() { + String collection = "test"; + String primaryKey = "id"; + Map confMap = new HashMap<>(); + confMap.put(SinkConfig.COLLECTION.key(), collection); + confMap.put(SinkConfig.PRIMARY_KEYS.key(), Arrays.asList(primaryKey)); + + ReadonlyConfig pluginConf = ReadonlyConfig.fromMap(confMap); + CollectionInfo collectionInfo = new CollectionInfo(collection, pluginConf); + SeaTunnelRowType schema = + new SeaTunnelRowType( + new String[] {primaryKey, "name"}, + new SeaTunnelDataType[] {STRING_TYPE, STRING_TYPE}); + TypesenseRowSerializer typesenseRowSerializer = + new TypesenseRowSerializer(collectionInfo, schema); + String id = "0001"; + String name = "jack"; + SeaTunnelRow row = new SeaTunnelRow(new Object[] {id, name}); + row.setRowKind(RowKind.UPDATE_AFTER); + Assertions.assertEquals(typesenseRowSerializer.serializeRowForDelete(row), id); + row.setRowKind(RowKind.INSERT); + String data = "{\"name\":\"jack\",\"id\":\"0001\"}"; + Assertions.assertEquals(typesenseRowSerializer.serializeRow(row), data); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java new file mode 100644 index 00000000000..568c96dbeed --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.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.typesense.sink; + +import org.apache.seatunnel.connectors.seatunnel.typesense.source.TypesenseSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TypesenseFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new TypesenseSourceFactory()).optionRule()); + Assertions.assertNotNull((new TypesenseSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java new file mode 100644 index 00000000000..716cb7cc20a --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java @@ -0,0 +1,35 @@ +/* + * 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.typesense.util; + +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class URLParamsConverterTest { + + @Test + public void convertParamsToJson() { + String json = URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10"); + Assertions.assertEquals(json, "{\"q\":\"*\",\"filter_by\":\"num_employees:10\"}"); + Assertions.assertThrows( + TypesenseConnectorException.class, + () -> URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10&b")); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 911750f7bf5..cf7314e619a 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -81,6 +81,7 @@ connector-activemq connector-qdrant connector-sls + connector-typesense diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java index 7052aa9bef8..a846949d857 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java @@ -75,6 +75,9 @@ public class RabbitmqIT extends TestSuiteBase implements TestResource { private static final String SINK_QUEUE_NAME = "test1"; private static final String USERNAME = "guest"; private static final String PASSWORD = "guest"; + private static final Boolean DURABLE = true; + private static final Boolean EXCLUSIVE = false; + private static final Boolean AUTO_DELETE = false; private static final Pair> TEST_DATASET = generateTestDataSet(); @@ -185,6 +188,9 @@ private void initRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); + config.setDurable(DURABLE); + config.setExclusive(EXCLUSIVE); + config.setAutoDelete(AUTO_DELETE); rabbitmqClient = new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); @@ -201,6 +207,9 @@ private RabbitmqClient initSinkRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); + config.setDurable(DURABLE); + config.setExclusive(EXCLUSIVE); + config.setAutoDelete(AUTO_DELETE); return new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf index b3a834bdc2f..61267a3adce 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf @@ -28,6 +28,9 @@ source { username = "guest" password = "guest" queue_name = "test" + durable = "true" + exclusive = "false" + auto_delete = "false" for_e2e_testing = true schema = { fields { @@ -61,6 +64,9 @@ sink { virtual_host = "/" username = "guest" password = "guest" + durable = "true" + exclusive = "false" + auto_delete = "false" queue_name = "test1" } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml new file mode 100644 index 00000000000..0a7243ed6a3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-typesense-e2e + SeaTunnel : E2E : Connector V2 : Typesense + + + 8 + 8 + UTF-8 + + + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-typesense + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java new file mode 100644 index 00000000000..afff4972e4b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java @@ -0,0 +1,265 @@ +/* + * 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.typesense; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.apache.commons.lang3.RandomUtils; + +import org.awaitility.Awaitility; +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.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; +import org.typesense.api.FieldTypes; +import org.typesense.model.Field; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class TypesenseIT extends TestSuiteBase implements TestResource { + + private static final String TYPESENSE_DOCKER_IMAGE = "typesense/typesense:26.0"; + + private static final String HOST = "e2e_typesense"; + + private static final int PORT = 8108; + + private GenericContainer typesenseServer; + + private TypesenseClient typesenseClient; + + private static final String sinkCollection = "typesense_test_collection"; + + private static final String sourceCollection = "typesense_test_collection_for_source"; + + @BeforeEach + @Override + public void startUp() throws Exception { + typesenseServer = + new GenericContainer<>(TYPESENSE_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withPrivilegedMode(true) + .withStartupAttempts(5) + .withCommand("--data-dir=/", "--api-key=xyz") + .withStartupTimeout(Duration.ofMinutes(5)) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(TYPESENSE_DOCKER_IMAGE))); + typesenseServer.setPortBindings(Lists.newArrayList(String.format("%s:%s", PORT, PORT))); + Startables.deepStart(Stream.of(typesenseServer)).join(); + log.info("Typesense container started"); + Awaitility.given() + .ignoreExceptions() + .atLeast(1L, TimeUnit.SECONDS) + .pollInterval(1L, TimeUnit.SECONDS) + .atMost(120L, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + private void initConnection() { + String host = typesenseServer.getContainerIpAddress(); + typesenseClient = + TypesenseClient.createInstance(Lists.newArrayList(host + ":8108"), "xyz", "http"); + } + + /** Test setting primary_keys parameter write Typesense */ + @TestTemplate + public void testFakeToTypesenseWithPrimaryKeys(TestContainer container) throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_primary_keys.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithRecreateSchema(TestContainer container) throws Exception { + List fields = new ArrayList<>(); + fields.add(new Field().name("T").type(FieldTypes.BOOL)); + Assertions.assertTrue(typesenseClient.createCollection(sinkCollection, fields)); + Map field = typesenseClient.getField(sinkCollection); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_recreate_schema.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + Assertions.assertNotEquals(field, typesenseClient.getField(sinkCollection)); + } + + @TestTemplate + public void testFakeToTypesenseWithErrorWhenNotExists(TestContainer container) + throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_error_when_not_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + + @TestTemplate + public void testFakeToTypesenseWithCreateWhenNotExists(TestContainer container) + throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_create_when_not_exists.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithDropData(TestContainer container) throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_drop_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithAppendData(TestContainer container) throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_append_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 6); + } + + @TestTemplate + public void testFakeToTypesenseWithErrorWhenDataExists(TestContainer container) + throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_error_when_data_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + + public List genTestData(int recordNum) { + ArrayList testDataList = new ArrayList<>(); + ObjectMapper objectMapper = new ObjectMapper(); + HashMap doc = new HashMap<>(); + for (int i = 0; i < recordNum; i++) { + try { + doc.put("num_employees", RandomUtils.nextInt()); + doc.put("flag", RandomUtils.nextBoolean()); + doc.put("num", RandomUtils.nextLong()); + doc.put("company_name", "A" + RandomUtils.nextInt(1, 100)); + testDataList.add(objectMapper.writeValueAsString(doc)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + return testDataList; + } + + @TestTemplate + public void testTypesenseSourceAndSink(TestContainer container) throws Exception { + int recordNum = 100; + List testData = genTestData(recordNum); + typesenseClient.createCollection(sourceCollection); + typesenseClient.insert(sourceCollection, testData); + Assertions.assertEquals( + typesenseClient.search(sourceCollection, null, 0).getFound(), recordNum); + Container.ExecResult execResult = container.executeJob("/typesense_source_and_sink.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(sinkCollection, null, 0).getFound(), recordNum); + } + + @TestTemplate + public void testTypesenseToTypesense(TestContainer container) throws Exception { + String typesenseToTypesenseSource = "typesense_to_typesense_source"; + String typesenseToTypesenseSink = "typesense_to_typesense_sink"; + List testData = new ArrayList<>(); + testData.add( + "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); + typesenseClient.createCollection(typesenseToTypesenseSource); + typesenseClient.insert(typesenseToTypesenseSource, testData); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 2); + Container.ExecResult execResult = container.executeJob("/typesense_to_typesense.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); + ObjectMapper objectMapper = new ObjectMapper(); + Map sourceData = objectMapper.readValue(testData.get(0), Map.class); + Map sinkData = + typesenseClient + .search(typesenseToTypesenseSink, null, 0) + .getHits() + .get(0) + .getDocument(); + Assertions.assertNotEquals(sourceData.remove("id"), sinkData.remove("id")); + Assertions.assertEquals(sourceData, sinkData); + } + + @TestTemplate + public void testTypesenseToTypesenseWithQuery(TestContainer container) throws Exception { + String typesenseToTypesenseSource = "typesense_to_typesense_source_with_query"; + String typesenseToTypesenseSink = "typesense_to_typesense_sink_with_query"; + List testData = new ArrayList<>(); + testData.add( + "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[18,97],\"c_int\":32,\"c_string\":\"String_48\"},\"company_name\":\"Company_9880\",\"company_name_list\":[\"Company_9880_Alias_1\",\"Company_9880_Alias_2\",\"Company_9880_Alias_3\",\"Company_9880_Alias_4\"],\"country\":\"Country_159\",\"id\":\"9880\",\"num_employees\":141}"); + typesenseClient.createCollection(typesenseToTypesenseSource); + typesenseClient.insert(typesenseToTypesenseSource, testData); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 3); + Container.ExecResult execResult = + container.executeJob("/typesense_to_typesense_with_query.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); + } + + @AfterEach + @Override + public void tearDown() { + typesenseServer.close(); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf new file mode 100644 index 00000000000..ab1c7b171dc --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.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 { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf new file mode 100644 index 00000000000..78870438e0a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf new file mode 100644 index 00000000000..01094580a62 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.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 { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + api_key = "xyz" + max_retry_count = 3 + max_batch_size = 10 + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf new file mode 100644 index 00000000000..6496ceb979c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.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 { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "ERROR_WHEN_DATA_EXISTS" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf new file mode 100644 index 00000000000..c3538c846a8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "ERROR_WHEN_SCHEMA_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf new file mode 100644 index 00000000000..2a767db02d2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf new file mode 100644 index 00000000000..ee7acce8a4b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "RECREATE_SCHEMA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf new file mode 100644 index 00000000000..25e63cf2443 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf @@ -0,0 +1,54 @@ +# +# 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 { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection_for_source" + api_key = "xyz" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + result_table_name = "typesense_test_table" + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf new file mode 100644 index 00000000000..f8c148a7afa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" + flink.execution.checkpointing.interval=5000 + flink.execution.restart.strategy = failure-rate + flink.execution.restart.failureInterval = 60000 + flink.execution.restart.failureRate = 100 + flink.execution.restart.delayInterval = 10000 + +} +source { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_source" + api_key = "xyz" + result_table_name = "typesense_test_table" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_sink" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf new file mode 100644 index 00000000000..7b069c90793 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" + flink.execution.checkpointing.interval=5000 + flink.execution.restart.strategy = failure-rate + flink.execution.restart.failureInterval = 60000 + flink.execution.restart.failureRate = 100 + flink.execution.restart.delayInterval = 10000 + +} +source { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_source_with_query" + api_key = "xyz" + query = "q=*&filter_by=c_row.c_int:>10" + result_table_name = "typesense_test_table" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 797ed87f0d0..28be63f3cf0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -78,6 +78,7 @@ connector-activemq-e2e connector-qdrant-e2e connector-sls-e2e + connector-typesense-e2e connector-email-e2e connector-cdc-opengauss-e2e diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml index ef801bdb9c0..99c75d324a8 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml @@ -112,12 +112,6 @@ ${flink.1.15.3.version} - - org.apache.flink - flink-runtime-web - ${flink.1.15.3.version} - - com.squareup.okhttp3 mockwebserver