From 0d08b200619bdc2017236c46d4134e56f095e738 Mon Sep 17 00:00:00 2001 From: dailai Date: Thu, 25 Jul 2024 10:00:13 +0800 Subject: [PATCH 01/32] [Improve][Connector-v2] Optimize the count table rows for jdbc-oracle and oracle-cdc (#7248) --- docs/en/connector-v2/source/Jdbc.md | 198 +++++------------- docs/en/connector-v2/source/Oracle-CDC.md | 40 ++++ .../cdc/oracle/config/OracleSourceConfig.java | 9 + .../config/OracleSourceConfigFactory.java | 16 ++ .../source/OracleIncrementalSource.java | 2 + .../OracleIncrementalSourceFactory.java | 2 + .../oracle/source/OracleSourceOptions.java | 12 ++ .../source/eumerator/OracleChunkSplitter.java | 6 +- .../cdc/oracle/utils/OracleUtils.java | 55 +++-- .../jdbc/config/JdbcSourceOptions.java | 12 ++ .../jdbc/config/JdbcSourceTableConfig.java | 8 + .../dialect/oracle/OracleDialect.java | 39 ++-- .../jdbc/source/JdbcSourceFactory.java | 4 + .../jdbc/source/JdbcSourceTable.java | 2 + .../jdbc/utils/JdbcCatalogUtils.java | 2 + .../seatunnel/cdc/oracle/OracleCDCIT.java | 31 ++- ..._console.conf => oraclecdc_to_oracle.conf} | 0 .../oraclecdc_to_oracle_skip_analysis.conf | 66 ++++++ .../oraclecdc_to_oracle_use_select_count.conf | 66 ++++++ .../seatunnel/jdbc/JdbcOracleIT.java | 28 ++- ...dbc_oracle_source_to_sink_use_select1.conf | 58 +++++ ...dbc_oracle_source_to_sink_use_select2.conf | 58 +++++ ...dbc_oracle_source_to_sink_use_select3.conf | 59 ++++++ 23 files changed, 594 insertions(+), 179 deletions(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/{oraclecdc_to_console.conf => oraclecdc_to_oracle.conf} (100%) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 7655083172a..7fab8d50b25 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -39,104 +39,32 @@ supports query SQL and can achieve projection effect. ## Options -| name | type | required | default value | -|--------------------------------------------|--------|----------|-----------------| -| url | String | Yes | - | -| driver | String | Yes | - | -| user | String | No | - | -| password | String | No | - | -| query | String | No | - | -| compatible_mode | String | No | - | -| connection_check_timeout_sec | Int | No | 30 | -| partition_column | String | No | - | -| partition_upper_bound | Long | No | - | -| partition_lower_bound | Long | No | - | -| partition_num | Int | No | job parallelism | -| fetch_size | Int | No | 0 | -| properties | Map | No | - | -| table_path | String | No | - | -| table_list | Array | No | - | -| where_condition | String | No | - | -| split.size | Int | No | 8096 | -| split.even-distribution.factor.lower-bound | Double | No | 0.05 | -| split.even-distribution.factor.upper-bound | Double | No | 100 | -| split.sample-sharding.threshold | Int | No | 1000 | -| split.inverse-sampling.rate | Int | No | 1000 | -| common-options | | No | - | - -### driver [string] - -The jdbc class name used to connect to the remote data source, if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. - -### user [string] - -userName - -### password [string] - -password - -### url [string] - -The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost/test - -### query [string] - -Query statement - -### compatible_mode [string] - -The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. - -### connection_check_timeout_sec [int] - -The time in seconds to wait for the database operation used to validate the connection to complete. - -### fetch_size [int] - -For queries that return a large number of objects, you can configure the row fetch size used in the query to -improve performance by reducing the number database hits required to satisfy the selection criteria. Zero means use jdbc default value. - -### properties - -Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. - -### table_path - -The path to the full path of table, you can use this configuration instead of `query`. - -examples: -- mysql: "testdb.table1" -- oracle: "test_schema.table1" -- sqlserver: "testdb.test_schema.table1" -- postgresql: "testdb.test_schema.table1" -- iris: "test_schema.table1" - -### table_list - -The list of tables to be read, you can use this configuration instead of `table_path` - -example - -```hocon -table_list = [ - { - table_path = "testdb.table1" - } - { - table_path = "testdb.table2" - query = "select * from testdb.table2 where id > 100" - } -] -``` - -### where_condition - -Common row filter conditions for all tables/queries, must start with `where`. for example `where id > 100` - -### common options - -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. +| name | type | required | default value | description | +|--------------------------------------------|---------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost/test | +| driver | String | Yes | - | The jdbc class name used to connect to the remote data source, if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. | +| user | String | No | - | userName | +| password | String | No | - | password | +| query | String | No | - | Query statement | +| compatible_mode | String | No | - | The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. | +| partition_column | String | No | - | The column name for split data. | +| partition_upper_bound | Long | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | +| partition_lower_bound | Long | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_num | Int | No | job parallelism | Not recommended for use, The correct approach is to control the number of split through `split.size`
How many splits do we need to split into, only support positive integer. default value is job parallelism. | +| use_select_count | Boolean | No | false | Use select count for table count rather then other methods in dynamic chunk split stage. This is currently only available for jdbc-oracle.In this scenario, select count directly is used when it is faster to update statistics using sql from analysis table | +| skip_analyze | Boolean | No | false | Skip the analysis of table count in dynamic chunk split stage. This is currently only available for jdbc-oracle.In this scenario, you schedule analysis table sql to update related table statistics periodically or your table data does not change frequently | +| fetch_size | Int | No | 0 | For queries that return a large number of objects, you can configure the row fetch size used in the query to improve performance by reducing the number database hits required to satisfy the selection criteria. Zero means use jdbc default value. | +| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. | +| table_path | String | No | - | The path to the full path of table, you can use this configuration instead of `query`.
examples:
`- mysql: "testdb.table1" `
`- oracle: "test_schema.table1" `
`- sqlserver: "testdb.test_schema.table1"`
`- postgresql: "testdb.test_schema.table1"`
`- iris: "test_schema.table1"` | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` | +| where_condition | String | No | - | Common row filter conditions for all tables/queries, must start with `where`. for example `where id > 100` | +| split.size | Int | No | 8096 | How many rows in one split, captured tables are split into multiple splits when read of table. | +| split.even-distribution.factor.lower-bound | Double | No | 0.05 | Not recommended for use.
The lower bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be greater than or equal to this lower bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is less, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 0.05. | +| split.even-distribution.factor.upper-bound | Double | No | 100 | Not recommended for use.
The upper bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be less than or equal to this upper bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is greater, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 100.0. | +| split.sample-sharding.threshold | Int | No | 1000 | This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. | +| split.inverse-sampling.rate | Int | No | 1000 | The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. | ## Parallel Reader @@ -152,50 +80,6 @@ The JDBC Source connector supports parallel reading of data from tables. SeaTunn * Number(int, bigint, decimal, ...) * Date -### Options Related To Split - -#### split.size - -How many rows in one split, captured tables are split into multiple splits when read of table. - -#### split.even-distribution.factor.lower-bound - -> Not recommended for use - -The lower bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be greater than or equal to this lower bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is less, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 0.05. - -#### split.even-distribution.factor.upper-bound - -> Not recommended for use - -The upper bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be less than or equal to this upper bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is greater, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 100.0. - -#### split.sample-sharding.threshold - -This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. - -#### split.inverse-sampling.rate - -The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. - -#### partition_column [string] - -The column name for split data. - -#### partition_upper_bound [BigDecimal] - -The partition_column max value for scan, if not set SeaTunnel will query database get max value. - -#### partition_lower_bound [BigDecimal] - -The partition_column min value for scan, if not set SeaTunnel will query database get min value. - -#### partition_num [int] - -> Not recommended for use, The correct approach is to control the number of split through `split.size` - -How many splits do we need to split into, only support positive integer. default value is job parallelism. - ## tips > If the table can not be split(for example, table have no Primary Key or Unique Index, and `partition_column` is not set), it will run in single concurrency. @@ -235,6 +119,35 @@ there are some reference value for params above. ### simple +#### Case 1 + +``` +Jdbc { + url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" + driver = "com.mysql.cj.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + query = "select * from type_bin" +} +``` + +#### Case 2 Use the select count(*) instead of analysis table for count table rows in dynamic chunk split stage + +``` +Jdbc { + url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" + driver = "com.mysql.cj.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + use_select_count = true + query = "select * from type_bin" +} +``` + +#### Case 3 Use the select NUM_ROWS from all_tables for the table rows but skip the analyze table. + ``` Jdbc { url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" @@ -242,6 +155,7 @@ Jdbc { connection_check_timeout_sec = 100 user = "root" password = "123456" + skip_analyze = true query = "select * from type_bin" } ``` diff --git a/docs/en/connector-v2/source/Oracle-CDC.md b/docs/en/connector-v2/source/Oracle-CDC.md index 2dfffedc66d..cedbda141f6 100644 --- a/docs/en/connector-v2/source/Oracle-CDC.md +++ b/docs/en/connector-v2/source/Oracle-CDC.md @@ -244,6 +244,8 @@ exit; | sample-sharding.threshold | Integer | No | 1000 | This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. | | inverse-sampling.rate | Integer | No | 1000 | The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. | | exactly_once | Boolean | No | false | Enable exactly once semantic. | +| use_select_count | Boolean | No | false | Use select count for table count rather then other methods in full stage.In this scenario, select count directly is used when it is faster to update statistics using sql from analysis table | +| skip_analyze | Boolean | No | false | Skip the analysis of table count in full stage.In this scenario, you schedule analysis table sql to update related table statistics periodically or your table data does not change frequently | | format | Enum | No | DEFAULT | Optional output format for Oracle CDC, valid enumerations are `DEFAULT`、`COMPATIBLE_DEBEZIUM_JSON`. | | debezium | Config | No | - | Pass-through [Debezium's properties](https://github.com/debezium/debezium/blob/v1.9.8.Final/documentation/modules/ROOT/pages/connectors/oracle.adoc#connector-properties) to Debezium Embedded Engine which is used to capture data changes from Oracle server. | | common-options | | no | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | @@ -270,6 +272,44 @@ source { } ``` +> Use the select count(*) instead of analysis table for count table rows in full stage +> +> ```conf +> source { +> # This is a example source plugin **only for test and demonstrate the feature source plugin** +> Oracle-CDC { +> result_table_name = "customers" +> use_select_count = true +> username = "system" +> password = "oracle" +> database-names = ["XE"] +> schema-names = ["DEBEZIUM"] +> table-names = ["XE.DEBEZIUM.FULL_TYPES"] +> base-url = "jdbc:oracle:thin:system/oracle@oracle-host:1521:xe" +> source.reader.close.timeout = 120000 +> } +> } +> ``` +> +> Use the select NUM_ROWS from all_tables for the table rows but skip the analyze table. +> +> ```conf +> source { +> # This is a example source plugin **only for test and demonstrate the feature source plugin** +> Oracle-CDC { +> result_table_name = "customers" +> skip_analyze = true +> username = "system" +> password = "oracle" +> database-names = ["XE"] +> schema-names = ["DEBEZIUM"] +> table-names = ["XE.DEBEZIUM.FULL_TYPES"] +> base-url = "jdbc:oracle:thin:system/oracle@oracle-host:1521:xe" +> source.reader.close.timeout = 120000 +> } +> } +> ``` + ### Support custom primary key for table ``` diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java index 5cdf1e9eecd..32bcb41f78f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java @@ -24,6 +24,7 @@ import io.debezium.config.Configuration; import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.relational.RelationalTableFilters; +import lombok.Getter; import java.util.List; import java.util.Properties; @@ -32,11 +33,17 @@ * Describes the connection information of the Oracle database and the configuration information for * performing snapshotting and streaming reading, such as splitSize. */ +@Getter public class OracleSourceConfig extends JdbcSourceConfig { private static final long serialVersionUID = 1L; + private final Boolean useSelectCount; + private final Boolean skipAnalyze; + public OracleSourceConfig( + Boolean useSelectCount, + Boolean skipAnalyze, StartupConfig startupConfig, StopConfig stopConfig, List databaseList, @@ -82,6 +89,8 @@ public OracleSourceConfig( connectMaxRetries, connectionPoolSize, exactlyOnce); + this.useSelectCount = useSelectCount; + this.skipAnalyze = skipAnalyze; } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java index 8e175bd7fe4..d6018083c29 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java @@ -38,6 +38,10 @@ public class OracleSourceConfigFactory extends JdbcSourceConfigFactory { private static final String DRIVER_CLASS_NAME = "oracle.jdbc.driver.OracleDriver"; private List schemaList; + + private Boolean useSelectCount; + + private Boolean skipAnalyze; /** * An optional list of regular expressions that match schema names to be monitored; any schema * name not included in the whitelist will be excluded from monitoring. By default all @@ -48,6 +52,16 @@ public JdbcSourceConfigFactory schemaList(List schemaList) { return this; } + public JdbcSourceConfigFactory useSelectCount(Boolean useSelectCount) { + this.useSelectCount = useSelectCount; + return this; + } + + public JdbcSourceConfigFactory skipAnalyze(Boolean skipAnalyze) { + this.skipAnalyze = skipAnalyze; + return this; + } + /** Creates a new {@link OracleSourceConfig} for the given subtask {@code subtaskId}. */ public OracleSourceConfig create(int subtask) { @@ -123,6 +137,8 @@ public OracleSourceConfig create(int subtask) { } return new OracleSourceConfig( + useSelectCount, + skipAnalyze, startupConfig, stopConfig, databaseList, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java index f3357f46e07..a1bbd0cb25c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java @@ -88,6 +88,8 @@ public SourceConfig.Factory createSourceConfigFactory(Readonly configFactory.startupOptions(startupConfig); configFactory.stopOptions(stopConfig); configFactory.schemaList(config.get(OracleSourceOptions.SCHEMA_NAMES)); + configFactory.useSelectCount(config.get(OracleSourceOptions.USE_SELECT_COUNT)); + configFactory.skipAnalyze(config.get(OracleSourceOptions.SKIP_ANALYZE)); configFactory.originUrl(config.get(JdbcCatalogOptions.BASE_URL)); return configFactory; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java index 2a0dc6b2907..21e08c2af7f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java @@ -61,6 +61,8 @@ public OptionRule optionRule() { JdbcCatalogOptions.BASE_URL, JdbcSourceOptions.DATABASE_NAMES, OracleSourceOptions.SCHEMA_NAMES, + OracleSourceOptions.USE_SELECT_COUNT, + OracleSourceOptions.SKIP_ANALYZE, JdbcSourceOptions.SERVER_TIME_ZONE, JdbcSourceOptions.CONNECT_TIMEOUT_MS, JdbcSourceOptions.CONNECT_MAX_RETRIES, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java index e6bbd77a410..f87ea1ccf5c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java @@ -53,4 +53,16 @@ public class OracleSourceOptions { .listType() .noDefaultValue() .withDescription("Schema name of the database to monitor."); + + public static final Option USE_SELECT_COUNT = + Options.key("use_select_count") + .booleanType() + .defaultValue(false) + .withDescription("Use select count for table count in full stage"); + + public static final Option SKIP_ANALYZE = + Options.key("skip_analyze") + .booleanType() + .defaultValue(false) + .withDescription("Skip the analysis of table count in full stage"); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java index 6525c3a2dbe..21cfebcd470 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.config.OracleSourceConfig; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.utils.OracleTypeUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.utils.OracleUtils; @@ -41,8 +42,11 @@ @Slf4j public class OracleChunkSplitter extends AbstractJdbcSourceChunkSplitter { + private final OracleSourceConfig oracleSourceConfig; + public OracleChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { super(sourceConfig, dialect); + this.oracleSourceConfig = (OracleSourceConfig) sourceConfig; } @Override @@ -80,7 +84,7 @@ public Object queryNextChunkMax( @Override public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { - return OracleUtils.queryApproximateRowCnt(jdbc, tableId); + return OracleUtils.queryApproximateRowCnt(oracleSourceConfig, jdbc, tableId); } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java index 1994bd6e03f..fbb3664be0a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.config.OracleSourceConfig; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.source.offset.RedoLogOffset; import org.apache.kafka.connect.source.SourceRecord; @@ -81,27 +82,41 @@ public static Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String }); } - public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) + public static long queryApproximateRowCnt( + OracleSourceConfig oracleSourceConfig, JdbcConnection jdbc, TableId tableId) throws SQLException { - final String analyzeTable = - String.format( - "analyze table %s compute statistics for table", - quoteSchemaAndTable(tableId)); - final String rowCountQuery = - String.format( - "select NUM_ROWS from all_tables where TABLE_NAME = '%s'", tableId.table()); - return jdbc.execute(analyzeTable) - .queryAndMap( - rowCountQuery, - rs -> { - if (!rs.next()) { - throw new SQLException( - String.format( - "No result returned after running query [%s]", - rowCountQuery)); - } - return rs.getLong(1); - }); + Boolean useSelectCount = oracleSourceConfig.getUseSelectCount(); + String rowCountQuery; + if (useSelectCount) { + rowCountQuery = String.format("select count(*) from %s", quoteSchemaAndTable(tableId)); + } else { + rowCountQuery = + String.format( + "select NUM_ROWS from all_tables where TABLE_NAME = '%s'", + tableId.table()); + Boolean skipAnalyze = oracleSourceConfig.getSkipAnalyze(); + if (!skipAnalyze) { + final String analyzeTable = + String.format( + "analyze table %s compute statistics for table", + quoteSchemaAndTable(tableId)); + // not skip analyze + log.info("analyze table sql: {}", analyzeTable); + jdbc.execute(analyzeTable); + } + } + log.info("row count query: {}", rowCountQuery); + return jdbc.queryAndMap( + rowCountQuery, + rs -> { + if (!rs.next()) { + throw new SQLException( + String.format( + "No result returned after running query [%s]", + rowCountQuery)); + } + return rs.getLong(1); + }); } public static Object queryMin( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java index 14ea5873350..6647d9c8eb1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java @@ -93,4 +93,16 @@ public interface JdbcSourceOptions { + "The value represents the denominator of the sampling rate fraction. " + "For example, a value of 1000 means a sampling rate of 1/1000. " + "This parameter is used when the sample sharding strategy is triggered."); + + Option USE_SELECT_COUNT = + Options.key("use_select_count") + .booleanType() + .defaultValue(false) + .withDescription("Use select count for table count"); + + Option SKIP_ANALYZE = + Options.key("skip_analyze") + .booleanType() + .defaultValue(false) + .withDescription("Skip the analysis of table count"); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java index b4a6e890dfc..d217a0b745a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java @@ -58,6 +58,12 @@ public class JdbcSourceTableConfig implements Serializable { @JsonProperty("partition_upper_bound") private BigDecimal partitionEnd; + @JsonProperty("use_select_count") + private Boolean useSelectCount; + + @JsonProperty("skip_analyze") + private Boolean skipAnalyze; + @Tolerate public JdbcSourceTableConfig() {} @@ -79,6 +85,8 @@ public static List of(ReadonlyConfig connectorConfig) { .partitionNumber(connectorConfig.get(JdbcOptions.PARTITION_NUM)) .partitionStart(connectorConfig.get(JdbcOptions.PARTITION_LOWER_BOUND)) .partitionEnd(connectorConfig.get(JdbcOptions.PARTITION_UPPER_BOUND)) + .useSelectCount(connectorConfig.get(JdbcSourceOptions.USE_SELECT_COUNT)) + .skipAnalyze(connectorConfig.get(JdbcSourceOptions.SKIP_ANALYZE)) .build(); tableList = Collections.singletonList(tableProperty); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java index 1bf14669490..e1aee7f7d88 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java @@ -180,34 +180,47 @@ public String tableIdentifier(TablePath tablePath) { public Long approximateRowCntStatement(Connection connection, JdbcSourceTable table) throws SQLException { - // 1. If no query is configured, use TABLE STATUS. - // 2. If a query is configured but does not contain a WHERE clause and tablePath is + // 1. Use select count + // 2. If no query is configured, use TABLE STATUS. + // 3. If a query is configured but does not contain a WHERE clause and tablePath is // configured, use TABLE STATUS. - // 3. If a query is configured with a WHERE clause, or a query statement is configured but + // 4. If a query is configured with a WHERE clause, or a query statement is configured but // tablePath is TablePath.DEFAULT, use COUNT(*). + String query = table.getQuery(); + boolean useTableStats = - StringUtils.isBlank(table.getQuery()) - || (!table.getQuery().toLowerCase().contains("where") + StringUtils.isBlank(query) + || (!query.toLowerCase().contains("where") && table.getTablePath() != null && !TablePath.DEFAULT .getFullName() .equals(table.getTablePath().getFullName())); + if (table.getUseSelectCount()) { + useTableStats = false; + if (StringUtils.isBlank(query)) { + query = "SELECT * FROM " + tableIdentifier(table.getTablePath()); + } + } + if (useTableStats) { TablePath tablePath = table.getTablePath(); - String analyzeTable = - String.format( - "analyze table %s compute statistics for table", - tableIdentifier(tablePath)); String rowCountQuery = String.format( "select NUM_ROWS from all_tables where OWNER = '%s' AND TABLE_NAME = '%s' ", tablePath.getSchemaName(), tablePath.getTableName()); - try (Statement stmt = connection.createStatement()) { - log.info("Split Chunk, approximateRowCntStatement: {}", analyzeTable); - stmt.execute(analyzeTable); + String analyzeTable = + String.format( + "analyze table %s compute statistics for table", + tableIdentifier(tablePath)); + if (!table.getSkipAnalyze()) { + log.info("Split Chunk, approximateRowCntStatement: {}", analyzeTable); + stmt.execute(analyzeTable); + } else { + log.warn("Skip analyze, approximateRowCntStatement: {}", analyzeTable); + } log.info("Split Chunk, approximateRowCntStatement: {}", rowCountQuery); try (ResultSet rs = stmt.executeQuery(rowCountQuery)) { if (!rs.next()) { @@ -220,7 +233,7 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta } } } - return SQLUtils.countForSubquery(connection, table.getQuery()); + return SQLUtils.countForSubquery(connection, query); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index 54e8d5173b3..b9ca90ed538 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -46,6 +46,7 @@ import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.QUERY; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.URL; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.USER; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SKIP_ANALYZE; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_INVERSE_SAMPLING_RATE; @@ -53,6 +54,7 @@ import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_SIZE; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.TABLE_LIST; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.TABLE_PATH; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.USE_SELECT_COUNT; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.WHERE_CONDITION; @Slf4j @@ -94,6 +96,8 @@ public OptionRule optionRule() { COMPATIBLE_MODE, PROPERTIES, QUERY, + USE_SELECT_COUNT, + SKIP_ANALYZE, TABLE_PATH, WHERE_CONDITION, TABLE_LIST, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java index fea73824720..8aad94c8b69 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java @@ -37,5 +37,7 @@ public class JdbcSourceTable implements Serializable { private final Integer partitionNumber; private final BigDecimal partitionStart; private final BigDecimal partitionEnd; + private final Boolean useSelectCount; + private final Boolean skipAnalyze; private final CatalogTable catalogTable; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java index 83d5bfa7692..a6896322065 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java @@ -90,6 +90,8 @@ public static Map getTables( .partitionNumber(tableConfig.getPartitionNumber()) .partitionStart(tableConfig.getPartitionStart()) .partitionEnd(tableConfig.getPartitionEnd()) + .useSelectCount(tableConfig.getUseSelectCount()) + .skipAnalyze(tableConfig.getSkipAnalyze()) .catalogTable(catalogTable) .build(); tables.put(tablePath, jdbcSourceTable); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java index 125d57915c2..0192fae3f70 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java @@ -137,7 +137,22 @@ public void startUp() throws Exception { @TestTemplate public void testOracleCdcCheckDataE2e(TestContainer container) throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle.conf", false); + } + + @TestTemplate + public void testOracleCdcCheckDataE2eForUseSelectCount(TestContainer container) + throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle_use_select_count.conf", false); + } + @TestTemplate + public void testOracleCdcCheckDataE2eForSkipAnalysis(TestContainer container) throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle_skip_analysis.conf", true); + } + + private void checkDataForTheJob( + TestContainer container, String jobConfPath, Boolean skipAnalysis) throws Exception { clearTable(DATABASE, SOURCE_TABLE1); clearTable(DATABASE, SOURCE_TABLE2); clearTable(DATABASE, SINK_TABLE1); @@ -145,10 +160,24 @@ public void testOracleCdcCheckDataE2e(TestContainer container) throws Exception insertSourceTable(DATABASE, SOURCE_TABLE1); + if (skipAnalysis) { + // analyzeTable before execute job + String analyzeTable = + String.format( + "analyze table " + + "\"DEBEZIUM\".\"FULL_TYPES\" " + + "compute statistics for table"); + log.info("analyze table {}", analyzeTable); + try (Connection connection = testConnection(ORACLE_CONTAINER); + Statement statement = connection.createStatement()) { + statement.execute(analyzeTable); + } + } + CompletableFuture.supplyAsync( () -> { try { - container.executeJob("/oraclecdc_to_console.conf"); + container.executeJob(jobConfPath); } catch (Exception e) { log.error("Commit task exception :" + e.getMessage()); throw new RuntimeException(e); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle.conf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_console.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf new file mode 100644 index 00000000000..233fc735ef6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + skip_analyze = true + username = "system" + password = "top_secret" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + # log.mining.strategy = "online_catalog" + # log.mining.continuous.mine = true + database.oracle.jdbc.timezoneAsRegion = "false" + } + } +} + +transform { +} + +sink { +Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "system" + password = "top_secret" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.SINK_FULL_TYPES" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 +} +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf new file mode 100644 index 00000000000..dd93cd4f9e1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + use_select_count = true + username = "system" + password = "top_secret" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + # log.mining.strategy = "online_catalog" + # log.mining.continuous.mine = true + database.oracle.jdbc.timezoneAsRegion = "false" + } + } +} + +transform { +} + +sink { +Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "system" + password = "top_secret" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.SINK_FULL_TYPES" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 +} +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java index 70c9d39cf45..9d3597c435b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java @@ -28,6 +28,7 @@ import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.OracleContainer; @@ -40,6 +41,7 @@ import java.math.BigDecimal; import java.sql.Date; +import java.sql.Statement; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; @@ -63,7 +65,11 @@ public class JdbcOracleIT extends AbstractJdbcIT { private static final String SINK_TABLE = "E2E_TABLE_SINK"; private static final String CATALOG_TABLE = "E2E_TABLE_CATALOG"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_oracle_source_to_sink.conf"); + Lists.newArrayList( + "/jdbc_oracle_source_to_sink.conf", + "/jdbc_oracle_source_to_sink_use_select1.conf", + "/jdbc_oracle_source_to_sink_use_select2.conf", + "/jdbc_oracle_source_to_sink_use_select3.conf"); private static final String CREATE_SQL = "create table %s\n" @@ -163,7 +169,7 @@ String driverUrl() { @Override Pair> initTestData() { List rows = new ArrayList<>(); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 20000; i++) { SeaTunnelRow row = new SeaTunnelRow( new Object[] { @@ -237,4 +243,22 @@ protected void initCatalog() { SCHEMA); catalog.open(); } + + @BeforeAll + @Override + public void startUp() { + super.startUp(); + // analyzeTable before execute job + String analyzeTable = + String.format( + "analyze table " + + quoteIdentifier(SOURCE_TABLE) + + " compute statistics for table"); + log.info("analyze table {}", analyzeTable); + try (Statement stmt = connection.createStatement()) { + stmt.execute(analyzeTable); + } catch (Exception e) { + log.error("Error when analyze table", e); + } + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf new file mode 100644 index 00000000000..8a0c8310443 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf @@ -0,0 +1,58 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = true + query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL FROM E2E_TABLE_SOURCE" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf new file mode 100644 index 00000000000..ebebdb55051 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf @@ -0,0 +1,58 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = true + table_path = TESTUSER.E2E_TABLE_SOURCE + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf new file mode 100644 index 00000000000..d00ce9b6434 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = false + skip_analyze = true + table_path = TESTUSER.E2E_TABLE_SOURCE + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} From 876d2f08d28313c963e520374b1d043f86d53302 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 25 Jul 2024 10:24:55 +0800 Subject: [PATCH 02/32] [Fix][Dist] Remove aws jar dependency in release package (#7257) * [Fix][Dist] Remove aws jar dependency in release package * [Fix][Dist] Remove aws jar dependency in release package --- seatunnel-dist/pom.xml | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index a5dd203f837..c96bf0b612b 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -702,7 +702,7 @@ com.amazonaws aws-java-sdk-bundle - 1.11.271 + ${aws-java-sdk.version} provided @@ -883,12 +883,6 @@ ${project.version} provided - - com.amazonaws - aws-java-sdk-bundle - ${aws-java-sdk.version} - provided - org.apache.seatunnel From d003bd85b65b307e77867781986cc7d58c3ba80a Mon Sep 17 00:00:00 2001 From: zhangdonghao <39961809+hawk9821@users.noreply.github.com> Date: Fri, 26 Jul 2024 10:44:09 +0800 Subject: [PATCH 03/32] [Feature][Zeta] Added the metrics information of table statistics in multi-table mode (#7212) --- .../sink/multitablesink/MultiTableSink.java | 5 + .../engine/e2e/MultiTableMetricsIT.java | 125 ++++++++++++++++++ .../batch_fake_multi_table_to_console.conf | 64 +++++++++ .../engine/client/SeaTunnelClientTest.java | 114 ++++++++++++++++ .../batch_fake_multi_table_to_console.conf | 66 +++++++++ .../rest/RestHttpGetCommandProcessor.java | 74 ++++++++++- .../server/task/SeaTunnelSourceCollector.java | 53 ++++++-- .../server/task/SourceSeaTunnelTask.java | 13 +- .../server/task/flow/SinkFlowLifeCycle.java | 38 ++++++ 9 files changed, 537 insertions(+), 15 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf create mode 100644 seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java index bb04283ca68..923ecff8b88 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.factory.MultiTableFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -149,6 +150,10 @@ public Optional> getCommitInfoSerializer() { return Optional.of(new MultiTableSinkAggregatedCommitter(aggCommitters)); } + public List getSinkTables() { + return sinks.keySet().stream().map(TablePath::of).collect(Collectors.toList()); + } + @Override public Optional> getAggregatedCommitInfoSerializer() { diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java new file mode 100644 index 00000000000..59942eb4cc8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.e2e; + +import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment; +import org.apache.seatunnel.engine.client.job.ClientJobProxy; +import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.JobConfig; +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.core.job.JobStatus; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.rest.RestConstant; + +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.Test; + +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static io.restassured.RestAssured.given; +import static org.hamcrest.Matchers.equalTo; + +public class MultiTableMetricsIT { + + private static final String HOST = "http://localhost:"; + + private static ClientJobProxy batchJobProxy; + + private static HazelcastInstanceImpl node1; + + private static SeaTunnelClient engineClient; + + @BeforeEach + void beforeClass() throws Exception { + String testClusterName = TestUtils.getClusterName("RestApiIT"); + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); + node1 = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); + + ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); + clientConfig.setClusterName(testClusterName); + engineClient = new SeaTunnelClient(clientConfig); + + String batchFilePath = TestUtils.getResource("batch_fake_multi_table_to_console.conf"); + JobConfig batchConf = new JobConfig(); + batchConf.setName("batch_fake_multi_table_to_console"); + ClientJobExecutionEnvironment batchJobExecutionEnv = + engineClient.createExecutionContext(batchFilePath, batchConf, seaTunnelConfig); + batchJobProxy = batchJobExecutionEnv.execute(); + Awaitility.await() + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.FINISHED, batchJobProxy.getJobStatus())); + } + + @Test + public void multiTableMetrics() { + Collections.singletonList(node1) + .forEach( + instance -> { + given().get( + HOST + + instance.getCluster() + .getLocalMember() + .getAddress() + .getPort() + + RestConstant.JOB_INFO_URL + + "/" + + batchJobProxy.getJobId()) + .then() + .statusCode(200) + .body("jobName", equalTo("batch_fake_multi_table_to_console")) + .body("jobStatus", equalTo("FINISHED")) + .body("metrics.SourceReceivedCount", equalTo("50")) + .body("metrics.SinkWriteCount", equalTo("50")) + .body( + "metrics.TableSourceReceivedCount.'fake.table1'", + equalTo("20")) + .body( + "metrics.TableSourceReceivedCount.'fake.public.table2'", + equalTo("30")) + .body( + "metrics.TableSinkWriteCount.'fake.table1'", + equalTo("20")) + .body( + "metrics.TableSinkWriteCount.'fake.public.table2'", + equalTo("30")); + }); + } + + @AfterEach + void afterClass() { + if (engineClient != null) { + engineClient.close(); + } + + if (node1 != null) { + node1.shutdown(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf new file mode 100644 index 00000000000..c51929a0edb --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake1" + row.num = 20 + schema = { + table = "fake.table1" + fields { + name = "string" + age = "int" + } + } + } + + FakeSource { + result_table_name = "fake2" + row.num = 30 + schema = { + table = "fake.public.table2" + fields { + name = "string" + age = "int" + sex = "int" + } + } + } +} + +transform { +} + +sink { + console { + source_table_name = "fake1" + } + console { + source_table_name = "fake2" + } +} diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java index d7e55db4ec2..100aa0b3203 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java +++ b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.engine.client; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.common.config.Common; @@ -51,10 +53,14 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Spliterators; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; @@ -548,6 +554,114 @@ public void testSavePointAndRestoreWithSavePoint() throws Exception { } } + @Test + public void testGetMultiTableJobMetrics() { + Common.setDeployMode(DeployMode.CLIENT); + String filePath = TestUtils.getResource("/batch_fake_multi_table_to_console.conf"); + JobConfig jobConfig = new JobConfig(); + jobConfig.setName("testGetMultiTableJobMetrics"); + + SeaTunnelClient seaTunnelClient = createSeaTunnelClient(); + JobClient jobClient = seaTunnelClient.getJobClient(); + + try { + ClientJobExecutionEnvironment jobExecutionEnv = + seaTunnelClient.createExecutionContext(filePath, jobConfig, SEATUNNEL_CONFIG); + + final ClientJobProxy clientJobProxy = jobExecutionEnv.execute(); + CompletableFuture objectCompletableFuture = + CompletableFuture.supplyAsync( + () -> { + return clientJobProxy.waitForJobComplete(); + }); + long jobId = clientJobProxy.getJobId(); + + await().atMost(30000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertTrue( + jobClient.getJobDetailStatus(jobId).contains("FINISHED") + && jobClient + .listJobStatus(true) + .contains("FINISHED"))); + + String jobMetrics = jobClient.getJobMetrics(jobId); + + Assertions.assertTrue(jobMetrics.contains(SOURCE_RECEIVED_COUNT + "#fake.table1")); + Assertions.assertTrue( + jobMetrics.contains(SOURCE_RECEIVED_COUNT + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.table1")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.public.table2")); + + log.info("jobMetrics : {}", jobMetrics); + JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); + List metricNameList = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize( + jobMetricsStr.fieldNames(), 0), + false) + .filter( + metricName -> + metricName.startsWith(SOURCE_RECEIVED_COUNT) + || metricName.startsWith(SINK_WRITE_COUNT)) + .collect(Collectors.toList()); + + Map totalCount = + metricNameList.stream() + .filter(metrics -> !metrics.contains("#")) + .collect( + Collectors.toMap( + metrics -> metrics, + metrics -> + StreamSupport.stream( + jobMetricsStr + .get(metrics) + .spliterator(), + false) + .mapToLong( + value -> + value.get("value") + .asLong()) + .sum())); + + Map tableCount = + metricNameList.stream() + .filter(metrics -> metrics.contains("#")) + .collect( + Collectors.toMap( + metrics -> metrics, + metrics -> + StreamSupport.stream( + jobMetricsStr + .get(metrics) + .spliterator(), + false) + .mapToLong( + value -> + value.get("value") + .asLong()) + .sum())); + + Assertions.assertEquals( + totalCount.get(SOURCE_RECEIVED_COUNT), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SOURCE_RECEIVED_COUNT)) + .mapToLong(Map.Entry::getValue) + .sum()); + Assertions.assertEquals( + totalCount.get(SINK_WRITE_COUNT), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SINK_WRITE_COUNT)) + .mapToLong(Map.Entry::getValue) + .sum()); + + } catch (ExecutionException | InterruptedException | JsonProcessingException e) { + throw new RuntimeException(e); + } finally { + seaTunnelClient.close(); + } + } + @AfterAll public static void after() { INSTANCE.shutdown(); diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf new file mode 100644 index 00000000000..df7ae51fe6e --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake1" + row.num = 20 + schema = { + table = "fake.table1" + fields { + name = "string" + age = "int" + } + } + parallelism = 1 + } + + FakeSource { + result_table_name = "fake2" + row.num = 30 + schema = { + table = "fake.public.table2" + fields { + name = "string" + age = "int" + sex = "int" + } + } + parallelism = 1 + } +} + +transform { +} + +sink { + console { + source_table_name = "fake1" + } + console { + source_table_name = "fake2" + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 6081b0f2eaf..d5d60b7cbb4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.common.metrics.JobMetrics; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.engine.common.Constant; @@ -64,8 +65,10 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.Spliterators; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; import static org.apache.seatunnel.engine.server.rest.RestConstant.FINISHED_JOBS_INFO; @@ -79,7 +82,9 @@ public class RestHttpGetCommandProcessor extends HttpCommandProcessor { private static final String SOURCE_RECEIVED_COUNT = "SourceReceivedCount"; + private static final String TABLE_SOURCE_RECEIVED_COUNT = "TableSourceReceivedCount"; private static final String SINK_WRITE_COUNT = "SinkWriteCount"; + private static final String TABLE_SINK_WRITE_COUNT = "TableSinkWriteCount"; private final Log4j2HttpGetCommandProcessor original; private NodeEngine nodeEngine; @@ -362,12 +367,31 @@ private void getRunningThread(HttpGetCommand command) { .collect(JsonArray::new, JsonArray::add, JsonArray::add)); } - private Map getJobMetrics(String jobMetrics) { - Map metricsMap = new HashMap<>(); + private Map getJobMetrics(String jobMetrics) { + Map metricsMap = new HashMap<>(); long sourceReadCount = 0L; long sinkWriteCount = 0L; + Map tableSourceReceivedCountMap = new HashMap<>(); + Map tableSinkWriteCountMap = new HashMap<>(); try { JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(jobMetricsStr.fieldNames(), 0), + false) + .filter(metricName -> metricName.contains("#")) + .forEach( + metricName -> { + String tableName = + TablePath.of(metricName.split("#")[1]).getFullName(); + if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { + tableSourceReceivedCountMap.put( + tableName, jobMetricsStr.get(metricName)); + } + if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { + tableSinkWriteCountMap.put( + tableName, jobMetricsStr.get(metricName)); + } + }); JsonNode sourceReceivedCountJson = jobMetricsStr.get(SOURCE_RECEIVED_COUNT); JsonNode sinkWriteCountJson = jobMetricsStr.get(SINK_WRITE_COUNT); for (int i = 0; i < jobMetricsStr.get(SOURCE_RECEIVED_COUNT).size(); i++) { @@ -379,9 +403,36 @@ private Map getJobMetrics(String jobMetrics) { } catch (JsonProcessingException | NullPointerException e) { return metricsMap; } + + Map tableSourceReceivedCount = + tableSourceReceivedCountMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.get("value").asLong()) + .sum())); + Map tableSinkWriteCount = + tableSinkWriteCountMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.get("value").asLong()) + .sum())); + metricsMap.put(SOURCE_RECEIVED_COUNT, sourceReadCount); metricsMap.put(SINK_WRITE_COUNT, sinkWriteCount); - + metricsMap.put(TABLE_SOURCE_RECEIVED_COUNT, tableSourceReceivedCount); + metricsMap.put(TABLE_SINK_WRITE_COUNT, tableSinkWriteCount); return metricsMap; } @@ -475,11 +526,24 @@ private JsonObject convertToJson(JobInfo jobInfo, long jobId) { .add( RestConstant.IS_START_WITH_SAVE_POINT, jobImmutableInformation.isStartWithSavePoint()) - .add(RestConstant.METRICS, JsonUtil.toJsonObject(getJobMetrics(jobMetrics))); + .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); return jobInfoJson; } + private JsonObject toJsonObject(Map jobMetrics) { + JsonObject members = new JsonObject(); + jobMetrics.forEach( + (key, value) -> { + if (value instanceof Map) { + members.add(key, toJsonObject((Map) value)); + } else { + members.add(key, value.toString()); + } + }); + return members; + } + private JsonObject getJobInfoJson(JobState jobState, String jobMetrics, JobDAGInfo jobDAGInfo) { return new JsonObject() .add(RestConstant.JOB_ID, String.valueOf(jobState.getJobId())) @@ -498,6 +562,6 @@ private JsonObject getJobInfoJson(JobState jobState, String jobMetrics, JobDAGIn DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) .add(RestConstant.JOB_DAG, JsonUtils.toJsonString(jobDAGInfo)) .add(RestConstant.PLUGIN_JARS_URLS, new JsonArray()) - .add(RestConstant.METRICS, JsonUtil.toJsonObject(getJobMetrics(jobMetrics))); + .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index f5d4aed1ab4..62612d0617a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.common.metrics.Meter; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventDispatcher; import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventHandler; @@ -34,12 +35,17 @@ import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; import org.apache.seatunnel.engine.server.task.flow.OneInputFlowLifeCycle; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + import lombok.extern.slf4j.Slf4j; import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; @@ -54,12 +60,16 @@ public class SeaTunnelSourceCollector implements Collector { private final List>> outputs; + private final MetricsContext metricsContext; + private final AtomicBoolean schemaChangeBeforeCheckpointSignal = new AtomicBoolean(false); private final AtomicBoolean schemaChangeAfterCheckpointSignal = new AtomicBoolean(false); private final Counter sourceReceivedCount; + private final Map sourceReceivedCountPerTable = new ConcurrentHashMap<>(); + private final Meter sourceReceivedQPS; private final Counter sourceReceivedBytes; @@ -77,17 +87,24 @@ public SeaTunnelSourceCollector( List>> outputs, MetricsContext metricsContext, FlowControlStrategy flowControlStrategy, - SeaTunnelDataType rowType) { + SeaTunnelDataType rowType, + List tablePaths) { this.checkpointLock = checkpointLock; this.outputs = outputs; this.rowType = rowType; + this.metricsContext = metricsContext; if (rowType instanceof MultipleRowType) { ((MultipleRowType) rowType) .iterator() - .forEachRemaining( - type -> { - this.rowTypeMap.put(type.getKey(), type.getValue()); - }); + .forEachRemaining(type -> this.rowTypeMap.put(type.getKey(), type.getValue())); + } + if (CollectionUtils.isNotEmpty(tablePaths)) { + tablePaths.forEach( + tablePath -> + sourceReceivedCountPerTable.put( + getFullName(tablePath), + metricsContext.counter( + SOURCE_RECEIVED_COUNT + "#" + getFullName(tablePath)))); } sourceReceivedCount = metricsContext.counter(SOURCE_RECEIVED_COUNT); sourceReceivedQPS = metricsContext.meter(SOURCE_RECEIVED_QPS); @@ -100,14 +117,12 @@ public SeaTunnelSourceCollector( public void collect(T row) { try { if (row instanceof SeaTunnelRow) { + String tableId = ((SeaTunnelRow) row).getTableId(); int size; if (rowType instanceof SeaTunnelRowType) { size = ((SeaTunnelRow) row).getBytesSize((SeaTunnelRowType) rowType); } else if (rowType instanceof MultipleRowType) { - size = - ((SeaTunnelRow) row) - .getBytesSize( - rowTypeMap.get(((SeaTunnelRow) row).getTableId())); + size = ((SeaTunnelRow) row).getBytesSize(rowTypeMap.get(tableId)); } else { throw new SeaTunnelEngineException( "Unsupported row type: " + rowType.getClass().getName()); @@ -115,6 +130,18 @@ public void collect(T row) { sourceReceivedBytes.inc(size); sourceReceivedBytesPerSeconds.markEvent(size); flowControlGate.audit((SeaTunnelRow) row); + if (StringUtils.isNotEmpty(tableId)) { + String tableName = getFullName(TablePath.of(tableId)); + Counter sourceTableCounter = sourceReceivedCountPerTable.get(tableName); + if (Objects.nonNull(sourceTableCounter)) { + sourceTableCounter.inc(); + } else { + Counter counter = + metricsContext.counter(SOURCE_RECEIVED_COUNT + "#" + tableName); + counter.inc(); + sourceReceivedCountPerTable.put(tableName, counter); + } + } } sendRecordToNext(new Record<>(row)); emptyThisPollNext = false; @@ -205,4 +232,12 @@ public void sendRecordToNext(Record record) throws IOException { } } } + + private String getFullName(TablePath tablePath) { + if (StringUtils.isBlank(tablePath.getTableName())) { + tablePath = + TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); + } + return tablePath.getFullName(); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java index 53171d40315..dbcde3e9d6e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlStrategy; import org.apache.seatunnel.engine.core.dag.actions.SourceAction; @@ -37,9 +39,11 @@ import lombok.Getter; import lombok.NonNull; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; public class SourceSeaTunnelTask extends SeaTunnelTask { @@ -76,10 +80,16 @@ public void init() throws Exception { + startFlowLifeCycle.getClass().getName()); } else { SeaTunnelDataType sourceProducedType; + List tablePaths = new ArrayList<>(); try { List producedCatalogTables = sourceFlow.getAction().getSource().getProducedCatalogTables(); sourceProducedType = CatalogTableUtil.convertToDataType(producedCatalogTables); + tablePaths = + producedCatalogTables.stream() + .map(CatalogTable::getTableId) + .map(TableIdentifier::toTablePath) + .collect(Collectors.toList()); } catch (UnsupportedOperationException e) { // TODO remove it when all connector use `getProducedCatalogTables` sourceProducedType = sourceFlow.getAction().getSource().getProducedType(); @@ -90,7 +100,8 @@ public void init() throws Exception { outputs, this.getMetricsContext(), FlowControlStrategy.fromMap(envOption), - sourceProducedType); + sourceProducedType, + tablePaths); ((SourceFlowLifeCycle) startFlowLifeCycle).setCollector(collector); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index 48c530a0c36..516e1c97c41 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -26,6 +26,8 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportResourceShare; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -43,6 +45,8 @@ import org.apache.seatunnel.engine.server.task.operation.sink.SinkRegisterOperation; import org.apache.seatunnel.engine.server.task.record.Barrier; +import org.apache.commons.lang3.StringUtils; + import com.hazelcast.cluster.Address; import lombok.extern.slf4j.Slf4j; @@ -52,9 +56,11 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -92,6 +98,8 @@ public class SinkFlowLifeCycle sinkWriteCountPerTable = new ConcurrentHashMap<>(); + private Meter sinkWriteQPS; private Counter sinkWriteBytes; @@ -125,6 +133,15 @@ public SinkFlowLifeCycle( sinkWriteQPS = metricsContext.meter(SINK_WRITE_QPS); sinkWriteBytes = metricsContext.counter(SINK_WRITE_BYTES); sinkWriteBytesPerSeconds = metricsContext.meter(SINK_WRITE_BYTES_PER_SECONDS); + if (sinkAction.getSink() instanceof MultiTableSink) { + List sinkTables = ((MultiTableSink) sinkAction.getSink()).getSinkTables(); + sinkTables.forEach( + tablePath -> + sinkWriteCountPerTable.put( + getFullName(tablePath), + metricsContext.counter( + SINK_WRITE_COUNT + "#" + getFullName(tablePath)))); + } } @Override @@ -256,6 +273,19 @@ public void received(Record record) { long size = ((SeaTunnelRow) record.getData()).getBytesSize(); sinkWriteBytes.inc(size); sinkWriteBytesPerSeconds.markEvent(size); + String tableId = ((SeaTunnelRow) record.getData()).getTableId(); + if (StringUtils.isNotBlank(tableId)) { + String tableName = getFullName(TablePath.of(tableId)); + Counter sinkTableCounter = sinkWriteCountPerTable.get(tableName); + if (Objects.nonNull(sinkTableCounter)) { + sinkTableCounter.inc(); + } else { + Counter counter = + metricsContext.counter(SINK_WRITE_COUNT + "#" + tableName); + counter.inc(); + sinkWriteCountPerTable.put(tableName, counter); + } + } } } } catch (Exception e) { @@ -315,4 +345,12 @@ public void restoreState(List actionStateList) throws Except ((SupportResourceShare) this.writer).setMultiTableResourceManager(resourceManager, 0); } } + + private String getFullName(TablePath tablePath) { + if (StringUtils.isBlank(tablePath.getTableName())) { + tablePath = + TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); + } + return tablePath.getFullName(); + } } From 31e59cdf82e727a68aab57235eb99dec129cefea Mon Sep 17 00:00:00 2001 From: Xuzz <90698333+Xuzhengz@users.noreply.github.com> Date: Fri, 26 Jul 2024 10:44:50 +0800 Subject: [PATCH 04/32] [Feature][Jdbc] Support hive compatibleMode add inceptor dialect (#7262) --- .../internal/dialect/DatabaseIdentifier.java | 1 + .../dialect/hive/HiveDialectFactory.java | 10 ++ .../dialect/inceptor/InceptorDialect.java | 35 +++++ .../inceptor/InceptorJdbcRowConverter.java | 141 ++++++++++++++++++ .../dialect/hive/HiveDialectFactoryTest.java | 36 +++++ 5 files changed, 223 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java index 17608392ff1..bf00298a742 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java @@ -40,4 +40,5 @@ public class DatabaseIdentifier { public static final String TIDB = "TiDB"; public static final String XUGU = "XUGU"; public static final String IRIS = "IRIS"; + public static final String INCEPTOR = "Inceptor"; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java index 56bd81b7f83..3ddf3bfab86 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor.InceptorDialect; import com.google.auto.service.AutoService; @@ -33,6 +34,15 @@ public boolean acceptsURL(String url) { @Override public JdbcDialect create() { + throw new UnsupportedOperationException( + "Can't create JdbcDialect without compatible mode for Hive"); + } + + @Override + public JdbcDialect create(String compatibleMode, String fieldId) { + if ("inceptor".equals(compatibleMode)) { + return new InceptorDialect(); + } return new HiveDialect(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java new file mode 100644 index 00000000000..9770fb63bdf --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.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.jdbc.internal.dialect.inceptor; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive.HiveDialect; + +public class InceptorDialect extends HiveDialect { + + @Override + public String dialectName() { + return DatabaseIdentifier.INCEPTOR; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new InceptorJdbcRowConverter(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java new file mode 100644 index 00000000000..806788b30eb --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor; + +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive.HiveJdbcRowConverter; + +import org.apache.commons.lang3.StringUtils; + +import java.math.BigDecimal; +import java.sql.PreparedStatement; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + +public class InceptorJdbcRowConverter extends HiveJdbcRowConverter { + + @Override + public String converterName() { + return DatabaseIdentifier.INCEPTOR; + } + + @Override + public PreparedStatement toExternal( + TableSchema tableSchema, SeaTunnelRow row, PreparedStatement statement) { + SeaTunnelRowType rowType = tableSchema.toPhysicalRowDataType(); + for (int fieldIndex = 0; fieldIndex < rowType.getTotalFields(); fieldIndex++) { + try { + SeaTunnelDataType seaTunnelDataType = rowType.getFieldType(fieldIndex); + int statementIndex = fieldIndex + 1; + Object fieldValue = row.getField(fieldIndex); + if (fieldValue == null) { + statement.setObject(statementIndex, StringUtils.EMPTY); + continue; + } + switch (seaTunnelDataType.getSqlType()) { + case STRING: + statement.setString(statementIndex, (String) row.getField(fieldIndex)); + break; + case BOOLEAN: + statement.setBoolean(statementIndex, (Boolean) row.getField(fieldIndex)); + break; + case TINYINT: + statement.setByte(statementIndex, (Byte) row.getField(fieldIndex)); + break; + case SMALLINT: + statement.setShort(statementIndex, (Short) row.getField(fieldIndex)); + break; + case INT: + statement.setInt(statementIndex, (Integer) row.getField(fieldIndex)); + break; + case BIGINT: + statement.setLong(statementIndex, (Long) row.getField(fieldIndex)); + break; + case FLOAT: + statement.setFloat(statementIndex, (Float) row.getField(fieldIndex)); + break; + case DOUBLE: + statement.setDouble(statementIndex, (Double) row.getField(fieldIndex)); + break; + case DECIMAL: + statement.setBigDecimal( + statementIndex, (BigDecimal) row.getField(fieldIndex)); + break; + case DATE: + LocalDate localDate = (LocalDate) row.getField(fieldIndex); + statement.setDate(statementIndex, java.sql.Date.valueOf(localDate)); + break; + case TIME: + writeTime(statement, statementIndex, (LocalTime) row.getField(fieldIndex)); + break; + case TIMESTAMP: + LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); + statement.setTimestamp( + statementIndex, java.sql.Timestamp.valueOf(localDateTime)); + break; + case BYTES: + statement.setBytes(statementIndex, (byte[]) row.getField(fieldIndex)); + break; + case NULL: + statement.setNull(statementIndex, java.sql.Types.NULL); + break; + case ARRAY: + SeaTunnelDataType elementType = + ((ArrayType) seaTunnelDataType).getElementType(); + Object[] array = (Object[]) row.getField(fieldIndex); + if (array == null) { + statement.setNull(statementIndex, java.sql.Types.ARRAY); + break; + } + if (SqlType.TINYINT.equals(elementType.getSqlType())) { + Short[] shortArray = new Short[array.length]; + for (int i = 0; i < array.length; i++) { + shortArray[i] = Short.valueOf(array[i].toString()); + } + statement.setObject(statementIndex, shortArray); + } else { + statement.setObject(statementIndex, array); + } + break; + case MAP: + case ROW: + default: + throw new JdbcConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + seaTunnelDataType); + } + } catch (Exception e) { + throw new JdbcConnectorException( + JdbcConnectorErrorCode.DATA_TYPE_CAST_FAILED, + "error field:" + rowType.getFieldNames()[fieldIndex], + e); + } + } + return statement; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java new file mode 100644 index 00000000000..169f51b6aea --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java @@ -0,0 +1,36 @@ +/* + * 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.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor.InceptorDialect; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class HiveDialectFactoryTest { + + @Test + public void testWithCompatibleMode() { + HiveDialectFactory hiveDialectFactory = new HiveDialectFactory(); + JdbcDialect inceptorDialect = hiveDialectFactory.create("inceptor", ""); + Assertions.assertTrue(inceptorDialect instanceof InceptorDialect); + JdbcDialect hiveDialect = hiveDialectFactory.create("", ""); + Assertions.assertTrue(hiveDialect instanceof HiveDialect); + } +} From d886495584c48392c2c09d2fb5d3d1780fc33244 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Fri, 26 Jul 2024 10:45:32 +0800 Subject: [PATCH 05/32] [Fix][Connector-V2] Fix doris TRANSFER_ENCODING header error (#7267) --- .../org/apache/seatunnel/connectors/doris/util/HttpUtil.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java index 46d1126c2fe..aa70d1f290a 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java @@ -21,6 +21,7 @@ import org.apache.http.impl.client.DefaultRedirectStrategy; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.RequestContent; /** util to build http client. */ public class HttpUtil { @@ -32,7 +33,8 @@ public class HttpUtil { protected boolean isRedirectable(String method) { return true; } - }); + }) + .addInterceptorLast(new RequestContent(true));; public CloseableHttpClient getHttpClient() { return httpClientBuilder.build(); From 3ccc6a8bd1e381a23e21eebcd9fca0d87694ba7c Mon Sep 17 00:00:00 2001 From: hailin0 Date: Mon, 29 Jul 2024 10:41:09 +0800 Subject: [PATCH 06/32] [Hotfix][Zeta] Fix task cannot be stopped when system is busy (#7280) --- .../seatunnel/engine/server/TaskExecutionService.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java index 19878545edc..94f0fa324fc 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java @@ -626,7 +626,7 @@ private void updateMetricsContextInImap() { if (localMap.size() > 0) { try { if (!metricsImap.tryLock( - Constant.IMAP_RUNNING_JOB_METRICS_KEY, 2, TimeUnit.SECONDS)) { + Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS)) { logger.warning("try lock failed in update metrics"); return; } @@ -640,7 +640,11 @@ private void updateMetricsContextInImap() { "The Imap acquisition failed due to the hazelcast node being offline or restarted, and will be retried next time", e); } finally { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + } catch (Throwable e) { + logger.warning("unlock imap failed in update metrics", e); + } } } this.printTaskExecutionRuntimeInfo(); From 21c4f5245e8777a9b3a967b6038d00d3f20d9fb8 Mon Sep 17 00:00:00 2001 From: corgy-w <73771213+corgy-w@users.noreply.github.com> Date: Mon, 29 Jul 2024 17:28:10 +0800 Subject: [PATCH 07/32] [Improve][Core] Improve base on plugin name of lookup strategy (#7278) * [bug][plugin-discovery] fix multi plugin discovery * [bug][plugin-discovery] optimize code --------- Co-authored-by: wangchao --- .../discovery/AbstractPluginDiscovery.java | 122 ++++++++++++++++-- .../SeaTunnelSourcePluginDiscoveryTest.java | 29 ++++- .../connectors/plugin-mapping.properties | 8 +- 3 files changed, 143 insertions(+), 16 deletions(-) diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java index 175ba435ed6..d4bd43c3d1c 100644 --- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java +++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java @@ -54,11 +54,13 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.ServiceLoader; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -430,17 +432,16 @@ public boolean accept(File pathname) { if (ArrayUtils.isEmpty(targetPluginFiles)) { return Optional.empty(); } - if (targetPluginFiles.length > 1) { - throw new IllegalArgumentException( - "Found multiple plugin jar: " - + Arrays.stream(targetPluginFiles) - .map(File::getPath) - .collect(Collectors.joining(",")) - + " for pluginIdentifier: " - + pluginIdentifier); - } try { - URL pluginJarPath = targetPluginFiles[0].toURI().toURL(); + URL pluginJarPath; + if (targetPluginFiles.length == 1) { + pluginJarPath = targetPluginFiles[0].toURI().toURL(); + } else { + pluginJarPath = + findMostSimlarPluginJarFile(targetPluginFiles, pluginJarPrefix) + .toURI() + .toURL(); + } log.info("Discovery plugin jar for: {} at: {}", pluginIdentifier, pluginJarPath); return Optional.of(pluginJarPath); } catch (MalformedURLException e) { @@ -451,4 +452,105 @@ public boolean accept(File pathname) { return Optional.empty(); } } + + private static File findMostSimlarPluginJarFile( + File[] targetPluginFiles, String pluginJarPrefix) { + String splitRegex = "\\-|\\_|\\."; + double maxSimlarity = -Integer.MAX_VALUE; + int mostSimlarPluginJarFileIndex = -1; + for (int i = 0; i < targetPluginFiles.length; i++) { + File file = targetPluginFiles[i]; + String fileName = file.getName(); + double similarity = + CosineSimilarityUtil.cosineSimilarity(pluginJarPrefix, fileName, splitRegex); + if (similarity > maxSimlarity) { + maxSimlarity = similarity; + mostSimlarPluginJarFileIndex = i; + } + } + return targetPluginFiles[mostSimlarPluginJarFileIndex]; + } + + static class CosineSimilarityUtil { + public static double cosineSimilarity(String textA, String textB, String splitRegrex) { + Set words1 = + new HashSet<>(Arrays.asList(textA.toLowerCase().split(splitRegrex))); + Set words2 = + new HashSet<>(Arrays.asList(textB.toLowerCase().split(splitRegrex))); + int[] termFrequency1 = calculateTermFrequencyVector(textA, words1, splitRegrex); + int[] termFrequency2 = calculateTermFrequencyVector(textB, words2, splitRegrex); + return calculateCosineSimilarity(termFrequency1, termFrequency2); + } + + private static int[] calculateTermFrequencyVector( + String text, Set words, String splitRegrex) { + int[] termFrequencyVector = new int[words.size()]; + String[] textArray = text.toLowerCase().split(splitRegrex); + List orderedWords = new ArrayList(); + words.clear(); + for (String word : textArray) { + if (!words.contains(word)) { + orderedWords.add(word); + words.add(word); + } + } + for (String word : textArray) { + if (words.contains(word)) { + int index = 0; + for (String w : orderedWords) { + if (w.equals(word)) { + termFrequencyVector[index]++; + break; + } + index++; + } + } + } + return termFrequencyVector; + } + + private static double calculateCosineSimilarity(int[] vectorA, int[] vectorB) { + double dotProduct = 0.0; + double magnitudeA = 0.0; + double magnitudeB = 0.0; + int vectorALength = vectorA.length; + int vectorBLength = vectorB.length; + if (vectorALength < vectorBLength) { + int[] vectorTemp = new int[vectorBLength]; + for (int i = 0; i < vectorB.length; i++) { + if (i <= vectorALength - 1) { + vectorTemp[i] = vectorA[i]; + } else { + vectorTemp[i] = 0; + } + } + vectorA = vectorTemp; + } + if (vectorALength > vectorBLength) { + int[] vectorTemp = new int[vectorALength]; + for (int i = 0; i < vectorA.length; i++) { + if (i <= vectorBLength - 1) { + vectorTemp[i] = vectorB[i]; + } else { + vectorTemp[i] = 0; + } + } + vectorB = vectorTemp; + } + for (int i = 0; i < vectorA.length; i++) { + dotProduct += vectorA[i] * vectorB[i]; + magnitudeA += Math.pow(vectorA[i], 2); + magnitudeB += Math.pow(vectorB[i], 2); + } + + magnitudeA = Math.sqrt(magnitudeA); + magnitudeB = Math.sqrt(magnitudeB); + + if (magnitudeA == 0 || magnitudeB == 0) { + return 0.0; // Avoid dividing by 0 + } else { + return dotProduct / (magnitudeA * magnitudeB); + } + } + } } diff --git a/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java b/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java index 81333d4b4df..88fd76d73be 100644 --- a/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java +++ b/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java @@ -32,10 +32,13 @@ import com.google.common.collect.Lists; import java.io.IOException; +import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; @DisabledOnOs(OS.WINDOWS) class SeaTunnelSourcePluginDiscoveryTest { @@ -47,7 +50,10 @@ class SeaTunnelSourcePluginDiscoveryTest { private static final List pluginJars = Lists.newArrayList( Paths.get(seatunnelHome, "connectors", "connector-http-jira.jar"), - Paths.get(seatunnelHome, "connectors", "connector-http.jar")); + Paths.get(seatunnelHome, "connectors", "connector-http.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka-alcs.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka-blcs.jar")); @BeforeEach public void before() throws IOException { @@ -67,12 +73,25 @@ void getPluginBaseClass() { List pluginIdentifiers = Lists.newArrayList( PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpJira"), - PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpBase")); + PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpBase"), + PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "Kafka"), + PluginIdentifier.of("seatunnel", PluginType.SINK.getType(), "Kafka-Blcs")); SeaTunnelSourcePluginDiscovery seaTunnelSourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery(); - Assertions.assertThrows( - IllegalArgumentException.class, - () -> seaTunnelSourcePluginDiscovery.getPluginJarPaths(pluginIdentifiers)); + Assertions.assertIterableEquals( + Stream.of( + Paths.get(seatunnelHome, "connectors", "connector-http-jira.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-http.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-kafka.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-kafka-blcs.jar") + .toString()) + .collect(Collectors.toList()), + seaTunnelSourcePluginDiscovery.getPluginJarPaths(pluginIdentifiers).stream() + .map(URL::getPath) + .collect(Collectors.toList())); } @AfterEach diff --git a/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties b/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties index be38939a7f0..ea20ad05b0f 100644 --- a/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties +++ b/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties @@ -18,4 +18,10 @@ seatunnel.source.HttpBase = connector-http seatunnel.sink.HttpBase = connector-http seatunnel.source.HttpJira = connector-http-jira -seatunnel.sink.HttpJira = connector-http-jira \ No newline at end of file +seatunnel.sink.HttpJira = connector-http-jira +seatunnel.source.Kafka = connector-kafka +seatunnel.sink.Kafka = connector-kafka +seatunnel.source.Kafka-Alcs = connector-kafka-alcs +seatunnel.sink.Kafka-Alcs = connector-kafka-alcs +seatunnel.source.Kafka-Blcs = connector-kafka-blcs +seatunnel.sink.Kafka-Blcs = connector-kafka-blcs \ No newline at end of file From 48974917082ccc08f196e892697f4c518a8b0ebf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=8D=A2=E5=AE=97=E6=9F=B1?= Date: Mon, 29 Jul 2024 17:29:23 +0800 Subject: [PATCH 08/32] fix [Bug] Unable to create a source for identifier 'Iceberg'. #7182 (#7279) Co-authored-by: luzongzhu --- seatunnel-connectors-v2/connector-doris/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/seatunnel-connectors-v2/connector-doris/pom.xml b/seatunnel-connectors-v2/connector-doris/pom.xml index fbaf9e1128b..85aafc97ad8 100644 --- a/seatunnel-connectors-v2/connector-doris/pom.xml +++ b/seatunnel-connectors-v2/connector-doris/pom.xml @@ -75,6 +75,12 @@ seatunnel-thrift-service ${project.version} optional + + + org.apache.thrift + libthrift + + org.apache.seatunnel From f012b2a6f093bcad49fbd59946e75ce6d6a97838 Mon Sep 17 00:00:00 2001 From: dailai Date: Mon, 29 Jul 2024 21:51:59 +0800 Subject: [PATCH 09/32] [Improve][Connector-v2] Optimize the way of databases and tables are checked for existence (#7261) --- .../common/exception/CommonError.java | 8 +++ .../common/exception/CommonErrorCode.java | 1 + .../jdbc/catalog/AbstractJdbcCatalog.java | 70 +++++++++++++++++-- .../jdbc/catalog/dm/DamengCatalog.java | 30 ++++---- .../jdbc/catalog/iris/IrisCatalog.java | 29 ++++---- .../jdbc/catalog/mysql/MySqlCatalog.java | 17 +++++ .../oceanbase/OceanBaseOracleCatalog.java | 33 +++++---- .../jdbc/catalog/oracle/OracleCatalog.java | 39 ++++++----- .../jdbc/catalog/psql/PostgresCatalog.java | 35 +++++----- .../catalog/redshift/RedshiftCatalog.java | 40 +++++------ .../jdbc/catalog/saphana/SapHanaCatalog.java | 30 ++++---- .../catalog/sqlserver/SqlServerCatalog.java | 29 ++++---- .../jdbc/catalog/xugu/XuguCatalog.java | 32 ++++----- .../jdbc/catalog/mysql/MySqlCatalogTest.java | 15 +++- .../catalog/oracle/OracleCatalogTest.java | 33 +++++++-- .../catalog/psql/PostgresCatalogTest.java | 32 +++++++-- .../sqlserver/SqlServerCatalogTest.java | 14 ++-- ...l_source_and_sink_with_multiple_tables.sql | 4 +- 18 files changed, 309 insertions(+), 182 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java index 782a071d011..e9adf4d70a1 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java @@ -285,4 +285,12 @@ public static SeaTunnelRuntimeException formatDateError(String date, String fiel params.put("field", field); return new SeaTunnelRuntimeException(CommonErrorCode.FORMAT_DATE_ERROR, params); } + + public static SeaTunnelRuntimeException unsupportedMethod( + String identifier, String methodName) { + Map params = new HashMap<>(); + params.put("identifier", identifier); + params.put("methodName", methodName); + return new SeaTunnelRuntimeException(CommonErrorCode.UNSUPPORTED_METHOD, params); + } } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java index 58939248482..79621c42168 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java @@ -77,6 +77,7 @@ public enum CommonErrorCode implements SeaTunnelErrorCode { FORMAT_DATETIME_ERROR( "COMMON-33", "The datetime format '' of field '' is not supported. Please check the datetime format."), + UNSUPPORTED_METHOD("COMMON-34", "'' unsupported the method ''"), ; private final String code; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index a033d0eaac7..8d0301b492e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -44,6 +44,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import lombok.extern.slf4j.Slf4j; + import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -63,11 +65,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_METHOD; +@Slf4j public abstract class AbstractJdbcCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcCatalog.class); protected static final Set SYS_DATABASES = new HashSet<>(); + protected static final Set EXCLUDED_SCHEMAS = new HashSet<>(); protected final String catalogName; protected final String defaultDatabase; @@ -259,6 +264,10 @@ protected String getListDatabaseSql() { throw new UnsupportedOperationException(); } + protected String getDatabaseWithConditionSql(String databaseName) { + throw CommonError.unsupportedMethod(this.catalogName, "getDatabaseWithConditionSql"); + } + @Override public List listDatabases() throws CatalogException { try { @@ -277,15 +286,35 @@ public List listDatabases() throws CatalogException { @Override public boolean databaseExists(String databaseName) throws CatalogException { - checkArgument(StringUtils.isNotBlank(databaseName)); - - return listDatabases().contains(databaseName); + if (StringUtils.isBlank(databaseName)) { + return false; + } + if (SYS_DATABASES.contains(databaseName)) { + return false; + } + try { + return querySQLResultExists( + getUrlFromDatabaseName(databaseName), + getDatabaseWithConditionSql(databaseName)); + } catch (SeaTunnelRuntimeException e) { + if (e.getSeaTunnelErrorCode().getCode().equals(UNSUPPORTED_METHOD.getCode())) { + log.warn( + "The catalog: {} is not supported the getDatabaseWithConditionSql for databaseExists", + this.catalogName); + return listDatabases().contains(databaseName); + } + throw e; + } } protected String getListTableSql(String databaseName) { throw new UnsupportedOperationException(); } + protected String getTableWithConditionSql(TablePath tablePath) { + throw CommonError.unsupportedMethod(this.catalogName, "getTableWithConditionSql"); + } + protected String getTableName(ResultSet rs) throws SQLException { String schemaName = rs.getString(1); String tableName = rs.getString(2); @@ -317,12 +346,28 @@ public List listTables(String databaseName) @Override public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()).contains(getTableName(tablePath)); - } catch (DatabaseNotExistException e) { + String databaseName = tablePath.getDatabaseName(); + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { return false; } + try { + return querySQLResultExists( + this.getUrlFromDatabaseName(databaseName), getTableWithConditionSql(tablePath)); + } catch (SeaTunnelRuntimeException e1) { + if (e1.getSeaTunnelErrorCode().getCode().equals(UNSUPPORTED_METHOD.getCode())) { + log.warn( + "The catalog: {} is not supported the getTableWithConditionSql for tableExists ", + this.catalogName); + try { + return databaseExists(tablePath.getDatabaseName()) + && listTables(tablePath.getDatabaseName()) + .contains(getTableName(tablePath)); + } catch (DatabaseNotExistException e2) { + return false; + } + } + throw e1; + } } @Override @@ -528,6 +573,17 @@ protected List queryString(String url, String sql, ResultSetConsumer listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java index 40f08dc50b5..02e58ea8573 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java @@ -57,7 +57,7 @@ public class IrisCatalog extends AbstractJdbcCatalog { private static final String LIST_TABLES_SQL_TEMPLATE = - "SELECT TABLE_SCHEMA,TABLE_NAME FROM INFORMATION_SCHEMA.Tables WHERE TABLE_SCHEMA='%s' and TABLE_TYPE != 'SYSTEM TABLE' and TABLE_TYPE != 'SYSTEM VIEW';"; + "SELECT TABLE_SCHEMA,TABLE_NAME FROM INFORMATION_SCHEMA.Tables WHERE TABLE_SCHEMA='%s' and TABLE_TYPE != 'SYSTEM TABLE' and TABLE_TYPE != 'SYSTEM VIEW'"; public IrisCatalog( String catalogName, String username, String password, JdbcUrlUtil.UrlInfo urlInfo) { @@ -101,13 +101,6 @@ protected String getTableName(ResultSet rs) throws SQLException { return schemaName + "." + tableName; } - // @Override - // protected String getSelectColumnsSql(TablePath tablePath) { - // return String.format( - // SELECT_COLUMNS_SQL_TEMPLATE, tablePath.getSchemaName(), - // tablePath.getTableName()); - // } - @Override protected Column buildColumn(ResultSet resultSet) throws SQLException { String columnName = resultSet.getString("COLUMN_NAME"); @@ -144,12 +137,24 @@ public boolean databaseExists(String databaseName) throws CatalogException { @Override public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return listTables(tablePath.getSchemaName()) - .contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { return false; } + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getSchemaName()) + " and TABLE_NAME = '%s'", + tablePath.getTableName()); + } + + @Override + protected String getUrlFromDatabaseName(String databaseName) { + return defaultUrl; } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java index 6b263b0fd46..e2df8ab24b9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java @@ -51,6 +51,12 @@ public class MySqlCatalog extends AbstractJdbcCatalog { private static final String SELECT_COLUMNS_SQL_TEMPLATE = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s' ORDER BY ORDINAL_POSITION ASC"; + private static final String SELECT_DATABASE_EXISTS = + "SELECT SCHEMA_NAME FROM information_schema.schemata WHERE SCHEMA_NAME = '%s'"; + + private static final String SELECT_TABLE_EXISTS = + "SELECT TABLE_SCHEMA,TABLE_NAME FROM information_schema.tables WHERE table_schema = '%s' AND table_name = '%s'"; + static { SYS_DATABASES.add("information_schema"); SYS_DATABASES.add("mysql"); @@ -68,6 +74,17 @@ public MySqlCatalog( this.typeConverter = new MySqlTypeConverter(version); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(SELECT_DATABASE_EXISTS, databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + SELECT_TABLE_EXISTS, tablePath.getDatabaseName(), tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SHOW DATABASES;"; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java index b4ece7db9c2..b98f4c4c2b2 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java @@ -25,8 +25,6 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; @@ -34,9 +32,10 @@ public class OceanBaseOracleCatalog extends OracleCatalog { static { - EXCLUDED_SCHEMAS = - Collections.unmodifiableList( - Arrays.asList("oceanbase", "LBACSYS", "ORAAUDITOR", "SYS")); + EXCLUDED_SCHEMAS.add("oceanbase"); + EXCLUDED_SCHEMAS.add("LBACSYS"); + EXCLUDED_SCHEMAS.add("ORAAUDITOR"); + EXCLUDED_SCHEMAS.add("SYS"); } public OceanBaseOracleCatalog( @@ -53,6 +52,21 @@ protected String getListDatabaseSql() { throw new UnsupportedOperationException(); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { + return false; + } + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)); + } + @Override public List listTables(String databaseName) throws CatalogException, DatabaseNotExistException { @@ -65,15 +79,6 @@ public List listTables(String databaseName) } } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return listTables(tablePath.getDatabaseName()).contains(getTableName(tablePath)); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java index b51369e3f58..1430cb387af 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,8 +28,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -46,7 +42,7 @@ @Slf4j public class OracleCatalog extends AbstractJdbcCatalog { - protected static List EXCLUDED_SCHEMAS = + protected static List EXCLUDED_SCHEMAS_ALL = Collections.unmodifiableList( Arrays.asList( "APPQOSSYS", @@ -101,6 +97,10 @@ public class OracleCatalog extends AbstractJdbcCatalog { + "ORDER BY \n" + " cols.column_id \n"; + static { + EXCLUDED_SCHEMAS.addAll(EXCLUDED_SCHEMAS_ALL); + } + public OracleCatalog( String catalogName, String username, @@ -110,6 +110,21 @@ public OracleCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where name = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return getListTableSql(tablePath.getDatabaseName()) + + " and OWNER = '" + + tablePath.getSchemaName() + + "' and table_name = '" + + tablePath.getTableName() + + "'"; + } + @Override protected String getListDatabaseSql() { return "SELECT name FROM v$database"; @@ -191,20 +206,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getSchemaAndTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java index 4697d1999ef..d5261e16d59 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,7 +29,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresTypeMapper; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import lombok.extern.slf4j.Slf4j; @@ -104,14 +102,28 @@ public PostgresCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where datname = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where table_schema = '%s' and table_name= '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { - return "select datname from pg_database;"; + return "select datname from pg_database"; } @Override protected String getListTableSql(String databaseName) { - return "SELECT table_schema, table_name FROM information_schema.tables;"; + return "SELECT table_schema, table_name FROM information_schema.tables"; } @Override @@ -231,21 +243,6 @@ protected void dropDatabaseInternal(String databaseName) throws CatalogException super.dropDatabaseInternal(databaseName); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - - return listTables(defaultDatabase).contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java index 7b29bbb8ea6..064b2473371 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -31,23 +30,17 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.redshift.RedshiftTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.redshift.RedshiftTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @Slf4j public class RedshiftCatalog extends AbstractJdbcCatalog { - protected static final Set EXCLUDED_SCHEMAS = new HashSet<>(4); - private final String SELECT_COLUMNS = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s' ORDER BY ordinal_position ASC"; @@ -80,6 +73,20 @@ public RedshiftCatalog( this.connectionMap = new ConcurrentHashMap<>(); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where datname = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where table_schema = '%s' and table_name = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override public void close() throws CatalogException { for (Map.Entry entry : connectionMap.entrySet()) { @@ -95,12 +102,12 @@ public void close() throws CatalogException { @Override protected String getListDatabaseSql() { - return "select datname from pg_database;"; + return "select datname from pg_database"; } @Override protected String getListTableSql(String databaseName) { - return "SELECT table_schema, table_name FROM information_schema.tables;"; + return "SELECT table_schema, table_name FROM information_schema.tables"; } @Override @@ -144,21 +151,6 @@ protected String getDropDatabaseSql(String databaseName) { return String.format("DROP DATABASE `%s`;", databaseName); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName().toLowerCase()); - } - return listTables(defaultDatabase) - .contains(tablePath.getSchemaAndTableName().toLowerCase()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override protected String getSelectColumnsSql(TablePath tablePath) { return String.format(SELECT_COLUMNS, tablePath.getSchemaName(), tablePath.getTableName()); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java index df6f4b3c248..19b8f668af9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java @@ -22,8 +22,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -31,8 +29,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -113,6 +109,18 @@ public SapHanaCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where SCHEMA_NAME = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + " and TABLE_NAME = '%s'", + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT SCHEMA_NAME FROM SCHEMAS"; @@ -203,20 +211,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java index 55660b36a2c..e4c63515220 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java @@ -22,7 +22,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -69,6 +68,20 @@ public SqlServerCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where name = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " and TABLE_SCHEMA = '%s' and TABLE_NAME = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT NAME FROM sys.databases"; @@ -147,20 +160,6 @@ protected String getUrlFromDatabaseName(String databaseName) { return baseUrl + ";databaseName=" + databaseName + ";" + suffix; } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables(defaultDatabase).contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java index 462e109c76a..a0b28e49abd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,8 +28,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -128,6 +124,20 @@ public XuguCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where DB_NAME = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where user_name = '%s' and table_name = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT DB_NAME FROM dba_databases"; @@ -210,20 +220,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getSchemaAndTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java index daf87b3693a..bc89d4c8c39 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerURLParser; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.MethodOrderer; @@ -39,7 +40,8 @@ class MySqlCatalogTest { static JdbcUrlUtil.UrlInfo sqlParse = SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); static JdbcUrlUtil.UrlInfo MysqlUrlInfo = - JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); + JdbcUrlUtil.getUrlInfo( + "jdbc:mysql://127.0.0.1:3306/test?useSSL=false&allowPublicKeyRetrieval=true"); static JdbcUrlUtil.UrlInfo pg = JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); static TablePath tablePathSQL; @@ -74,13 +76,22 @@ static void before() { tablePathPG = TablePath.of(databaseName, "pg_to_mysql"); tablePathOracle = TablePath.of(databaseName, "oracle_to_mysql"); sqlServerCatalog = new SqlServerCatalog("sqlserver", "sa", "root@123", sqlParse, null); - mySqlCatalog = new MySqlCatalog("mysql", "root", "root@123", MysqlUrlInfo); + mySqlCatalog = new MySqlCatalog("mysql", "root", "123456", MysqlUrlInfo); postgresCatalog = new PostgresCatalog("postgres", "postgres", "postgres", pg, null); mySqlCatalog.open(); sqlServerCatalog.open(); postgresCatalog.open(); } + @Test + void exists() { + Assertions.assertTrue(mySqlCatalog.databaseExists("test")); + Assertions.assertTrue(mySqlCatalog.tableExists(TablePath.of("test", "MY_TABLE"))); + Assertions.assertTrue(mySqlCatalog.tableExists(TablePath.of("test", "my_table"))); + Assertions.assertFalse(mySqlCatalog.tableExists(TablePath.of("test", "test"))); + Assertions.assertFalse(mySqlCatalog.databaseExists("mysql")); + } + @Test @Order(1) void getTable() { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java index 1c5fb5a2b22..75b22ec24dc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -27,17 +29,24 @@ @Disabled("Please Test it in your local environment") class OracleCatalogTest { - @Test - void testCatalog() { - OracleCatalog catalog = + + static OracleCatalog catalog; + + @BeforeAll + static void before() { + catalog = new OracleCatalog( "oracle", - "test", - "oracle", - OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521:xe"), + "c##gguser", + "testdb", + OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521/CDC_PDB"), null); catalog.open(); + } + + @Test + void testCatalog() { List strings = catalog.listDatabases(); @@ -45,4 +54,16 @@ void testCatalog() { catalog.createTable(new TablePath("XE", "TEST", "TEST003"), table, false); } + + @Test + void exist() { + Assertions.assertTrue(catalog.databaseExists("ORCLCDB")); + Assertions.assertTrue(catalog.tableExists(TablePath.of("ORCLCDB", "C##GGUSER", "myTable"))); + Assertions.assertFalse(catalog.databaseExists("ORCL")); + Assertions.assertTrue( + catalog.tableExists( + TablePath.of("ORCLCDB", "CDC_PDB", "ads_index_public_health_data"))); + Assertions.assertTrue( + catalog.tableExists(TablePath.of("ORCLCDB", "CDC_PDB", "ADS_INDEX_DISEASE_DATA"))); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java index c04c1941b0b..05a013ef691 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -31,15 +33,23 @@ @Slf4j class PostgresCatalogTest { - @Test - void testCatalog() { - JdbcUrlUtil.UrlInfo urlInfo = - JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); - PostgresCatalog catalog = - new PostgresCatalog("postgres", "postgres", "postgres", urlInfo, null); + static PostgresCatalog catalog; + + @BeforeAll + static void before() { + catalog = + new PostgresCatalog( + "postgres", + "pg", + "pg#2024", + JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/postgres"), + null); catalog.open(); + } + @Test + void testCatalog() { MySqlCatalog mySqlCatalog = new MySqlCatalog( "mysql", @@ -59,4 +69,14 @@ void testCatalog() { catalog.createTable( new TablePath("liulitest", "public", "all_types_table_02"), table, false); } + + @Test + void exists() { + Assertions.assertFalse(catalog.databaseExists("postgres")); + Assertions.assertFalse( + catalog.tableExists(TablePath.of("postgres", "pg_catalog", "pg_aggregate"))); + Assertions.assertTrue(catalog.databaseExists("zdykdb")); + Assertions.assertTrue( + catalog.tableExists(TablePath.of("zdykdb", "pg_catalog", "pg_class"))); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java index ea305ca0c1f..a18cc4abd9d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.MethodOrderer; @@ -38,7 +39,7 @@ class SqlServerCatalogTest { static JdbcUrlUtil.UrlInfo sqlParse = - SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); + SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1433;database=master"); static JdbcUrlUtil.UrlInfo MysqlUrlInfo = JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); static JdbcUrlUtil.UrlInfo pg = @@ -84,9 +85,14 @@ void listTables() { } @Test - void tableExists() { - - // boolean b = sqlServerCatalog.tableExists(tablePath); + void exists() { + Assertions.assertTrue(sqlServerCatalog.databaseExists("master")); + Assertions.assertTrue( + sqlServerCatalog.tableExists( + TablePath.of("master", "dbo", "MSreplication_options"))); + Assertions.assertTrue( + sqlServerCatalog.tableExists(TablePath.of("master", "dbo", "spt_fallback_db"))); + Assertions.assertFalse(sqlServerCatalog.tableExists(TablePath.of("master", "dbo", "xxx"))); } @Test diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql index a9b02e2ae3a..8c624959f87 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql @@ -55,10 +55,10 @@ CREATE TABLE sink_table WITH ( 'user' = 'root', 'password' = 'Abc!@#135_seatunnel', 'generate_sink_sql' = 'true', - 'database' = 'sink' + 'database' = 'sink', 'table' = '${table_name}' ); -- If it's multi-table synchronization, there's no need to set select columns. -- You can directly use the syntax 'INSERT INTO sink_table SELECT source_table'. -INSERT INTO sink_table SELECT source_table; \ No newline at end of file +INSERT INTO sink_table SELECT source_table; From ec1c3198bcffb8ff6b41014c36d4ddacedb03868 Mon Sep 17 00:00:00 2001 From: "Nothing." Date: Mon, 29 Jul 2024 21:53:01 +0800 Subject: [PATCH 10/32] [Docs]translate event-listener doc into chinese (#7274) --- docs/zh/concept/event-listener.md | 114 ++++++++++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 docs/zh/concept/event-listener.md diff --git a/docs/zh/concept/event-listener.md b/docs/zh/concept/event-listener.md new file mode 100644 index 00000000000..69972cbfc56 --- /dev/null +++ b/docs/zh/concept/event-listener.md @@ -0,0 +1,114 @@ +# 事件监听器 + +## 介绍 + +SeaTunnel提供了丰富的事件监听器功能,用于管理数据同步时的状态。此功能在需要监听任务运行状态时十分重要(`org.apache.seatunnel.api.event`)。本文档将指导您如何使用这些参数并有效地利用他们。 + +## 支持的引擎 + +> SeaTunnel Zeta
+> Flink
+> Spark
+ +## API + +事件(event)API的定义在 `org.apache.seatunnel.api.event`包中。 + +### Event Data API + +- `org.apache.seatunnel.api.event.Event` - 事件数据的接口。 +- `org.apache.seatunnel.api.event.EventType` - 事件数据的枚举值。 + +### Event Listener API + +您可以自定义事件处理器,例如将事件发送到外部系统。 + +- `org.apache.seatunnel.api.event.EventHandler` - 事件处理器的接口,SPI将会自动从类路径中加载子类。 + +### Event Collect API + +- `org.apache.seatunnel.api.source.SourceSplitEnumerator` - 在`SourceSplitEnumerator`加载事件监听器。 + +```java +package org.apache.seatunnel.api.source; + +public interface SourceSplitEnumerator { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this enumerator. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +- `org.apache.seatunnel.api.source.SourceReader` - 在`SourceReader`加载事件监听器。 + +```java +package org.apache.seatunnel.api.source; + +public interface SourceReader { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this reader. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +- `org.apache.seatunnel.api.sink.SinkWriter` - 在`SinkWriter`加载事件监听器。 + +```java +package org.apache.seatunnel.api.sink; + +public interface SinkWriter { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this writer. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +## 设置监听器 + +您需要设置引擎配置以使用事件监听器功能。 + +### Zeta 引擎 + +配置样例(seatunnel.yaml): + +``` +seatunnel: + engine: + event-report-http: + url: "http://example.com:1024/event/report" + headers: + Content-Type: application/json +``` + +### Flink 引擎 + +您可以定义 `org.apache.seatunnel.api.event.EventHandler` 接口并添加到类路径,SPI会自动加载。 + +支持的flink版本: 1.14.0+ + +样例: `org.apache.seatunnel.api.event.LoggingEventHandler` + +### Spark 引擎 + +您可以定义 `org.apache.seatunnel.api.event.EventHandler` 接口并添加到类路径,SPI会自动加载。 From b9acb573b75055931ab5dc911aa05a04ea830c4f Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 30 Jul 2024 16:12:57 +0800 Subject: [PATCH 11/32] [Feature][Transforms] Support cast to bytes function of sql (#7284) --- docs/en/transform-v2/sql-functions.md | 2 +- docs/zh/transform-v2/sql-functions.md | 2 +- .../src/test/resources/sql_transform/func_system.conf | 11 ++++++++++- .../seatunnel/transform/sql/zeta/ZetaSQLType.java | 4 ++++ .../transform/sql/zeta/functions/SystemFunction.java | 3 +++ 5 files changed, 19 insertions(+), 3 deletions(-) diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index e1c541ef1c9..3438a24de9c 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -889,7 +889,7 @@ CALL FROM_UNIXTIME(1672502400, 'yyyy-MM-dd HH:mm:ss','UTC+6') Converts a value to another data type. -Supported data types: STRING | VARCHAR, INT | INTEGER, LONG | BIGINT, BYTE, FLOAT, DOUBLE, DECIMAL(p,s), TIMESTAMP, DATE, TIME +Supported data types: STRING | VARCHAR, INT | INTEGER, LONG | BIGINT, BYTE, FLOAT, DOUBLE, DECIMAL(p,s), TIMESTAMP, DATE, TIME, BYTES Example: diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index cd90b948674..57c440a39b3 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -880,7 +880,7 @@ CALL FROM_UNIXTIME(1672502400, 'yyyy-MM-dd HH:mm:ss','UTC+6') 将一个值转换为另一个数据类型。 -支持的数据类型有:STRING | VARCHAR,INT | INTEGER,LONG | BIGINT,BYTE,FLOAT,DOUBLE,DECIMAL(p,s),TIMESTAMP,DATE,TIME +支持的数据类型有:STRING | VARCHAR,INT | INTEGER,LONG | BIGINT,BYTE,FLOAT,DOUBLE,DECIMAL(p,s),TIMESTAMP,DATE,TIME,BYTES 示例: diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index 558d0cceb38..14f41665e34 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6 from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7 from fake" } } @@ -155,6 +155,15 @@ sink { field_value = [ {equals_to = "23:51:09"} ] + }, + { + field_name = "c7" + field_type = "bytes" + field_value = [ + { + rule_type = NOT_NULL + } + ] } ] } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java index 934cd883080..45b269bae67 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.table.type.SqlType; @@ -69,6 +70,7 @@ public class ZetaSQLType { public static final String BIGINT = "BIGINT"; public static final String LONG = "LONG"; public static final String BYTE = "BYTE"; + public static final String BYTES = "BYTES"; public static final String DOUBLE = "DOUBLE"; public static final String FLOAT = "FLOAT"; public static final String TIMESTAMP = "TIMESTAMP"; @@ -311,6 +313,8 @@ private SeaTunnelDataType getCastType(CastExpression castExpression) { return BasicType.LONG_TYPE; case BYTE: return BasicType.BYTE_TYPE; + case BYTES: + return PrimitiveByteArrayType.INSTANCE; case DOUBLE: return BasicType.DOUBLE_TYPE; case FLOAT: diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java index 0039f0cade9..0b616b0fbe8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java @@ -24,6 +24,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -98,6 +99,8 @@ public static Object castAs(List args) { return Long.parseLong(v1.toString()); case "BYTE": return Byte.parseByte(v1.toString()); + case "BYTES": + return v1.toString().getBytes(StandardCharsets.UTF_8); case "DOUBLE": return Double.parseDouble(v1.toString()); case "FLOAT": From 9df557cb12d39831a737658994e2fe2662230e1a Mon Sep 17 00:00:00 2001 From: lizhenglei <127465317+jackyyyyyssss@users.noreply.github.com> Date: Tue, 30 Jul 2024 21:37:04 +0800 Subject: [PATCH 12/32] [Improve][Transform] Improve DynamicCompile transform (#7264) --- docs/en/transform-v2/dynamic-compile.md | 30 ++++++- .../e2e/common/container/TestContainer.java | 2 + .../flink/AbstractTestFlinkContainer.java | 7 ++ .../ConnectorPackageServiceContainer.java | 6 ++ .../seatunnel/SeaTunnelContainer.java | 6 ++ .../spark/AbstractTestSparkContainer.java | 7 ++ .../e2e/transform/TestDynamicCompileIT.java | 34 ++++++-- ...dynamic_groovy_java_compile_transform.conf | 2 + ...iple_dynamic_groovy_compile_transform.conf | 2 + ...ltiple_dynamic_java_compile_transform.conf | 2 + ...ngle_dynamic_groovy_compile_transform.conf | 1 + ...single_dynamic_java_compile_transform.conf | 1 + .../conf/single_groovy_path_compile.conf | 85 ++++++++++++++++++ .../conf/single_java_path_compile.conf | 86 +++++++++++++++++++ .../dynamic_compile/source_file/GroovyFile | 42 +++++++++ .../dynamic_compile/source_file/JavaFile | 39 +++++++++ .../dynamiccompile/CompilePattern.java | 23 +++++ .../DynamicCompileTransform.java | 30 +++++-- .../DynamicCompileTransformConfig.java | 12 +++ .../DynamicCompileTransformFactory.java | 8 ++ .../dynamiccompile/parse/AbstractParse.java | 2 +- .../{ParseUtil.java => AbstractParser.java} | 2 +- .../parse/GroovyClassParse.java | 4 +- ...yClassUtil.java => GroovyClassParser.java} | 4 +- .../dynamiccompile/parse/JavaClassParse.java | 4 +- ...avaClassUtil.java => JavaClassParser.java} | 27 +++--- 26 files changed, 432 insertions(+), 36 deletions(-) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/mixed_dynamic_groovy_java_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/multiple_dynamic_groovy_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/multiple_dynamic_java_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/single_dynamic_groovy_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/single_dynamic_java_compile_transform.conf (99%) create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{ParseUtil.java => AbstractParser.java} (97%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{GroovyClassUtil.java => GroovyClassParser.java} (89%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{JavaClassUtil.java => JavaClassParser.java} (72%) diff --git a/docs/en/transform-v2/dynamic-compile.md b/docs/en/transform-v2/dynamic-compile.md index 5bfbbadbe08..4a772e8cbf0 100644 --- a/docs/en/transform-v2/dynamic-compile.md +++ b/docs/en/transform-v2/dynamic-compile.md @@ -11,8 +11,10 @@ If the conversion is too complex, it may affect performance | name | type | required | default value | |------------------|--------|----------|---------------| -| source_code | string | yes | | -| compile_language | string | yes | | +| source_code | string | no | | +| compile_language | Enum | yes | | +| compile_pattern | Enum | no | SOURCE_CODE | +| absolute_path | string | no | | ### source_code [string] @@ -24,11 +26,20 @@ If there are third-party dependency packages, please place them in ${SEATUNNEL_H Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details -### compile_language [string] +### compile_language [Enum] Some syntax in Java may not be supported, please refer https://github.com/janino-compiler/janino GROOVY,JAVA +### compile_pattern [Enum] + +SOURCE_CODE,ABSOLUTE_PATH +If it is a SOURCE-CODE enumeration; the SOURCE-CODE attribute is required, and the ABSOLUTE_PATH enumeration;ABSOLUTE_PATH attribute is required + +### absolute_path [string] + +The absolute path of Java or Groovy files on the server + ## Example The data read from source is a table like this: @@ -46,6 +57,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor @@ -82,6 +94,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -113,6 +126,17 @@ transform { } } + + transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="GROOVY" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/GroovyFile""" + + } +} ``` Then the data in result table `fake1` will like this diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java index 33b196eeba0..07fef2c295a 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java @@ -55,4 +55,6 @@ default Container.ExecResult restoreJob(String confFile, String jobId) } String getServerLogs(); + + void copyFileToContainer(String path, String targetPath); } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java index 7145da6242d..ff16c0c7541 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.e2e.common.container.AbstractTestContainer; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -168,4 +169,10 @@ public String executeJobManagerInnerCommand(String command) throws IOException, InterruptedException { return jobManager.execInContainer("bash", "-c", command).getStdout(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, jobManager); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java index 4f5ea990295..3a27d78d423 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java @@ -234,4 +234,10 @@ public Container.ExecResult executeJob(String confFile, List variables) public String getServerLogs() { return server1.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, server1); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java index 2d9e76ea3b5..802b1c32fba 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java @@ -455,4 +455,10 @@ public Container.ExecResult restoreJob(String confFile, String jobId) public String getServerLogs() { return server.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, server); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java index fe07d082afd..9970ffb3aa7 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.e2e.common.container.AbstractTestContainer; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -118,4 +119,10 @@ public Container.ExecResult executeJob(String confFile, List variables) public String getServerLogs() { return master.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, master); + } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java index 5c5e69dad25..b57b332353a 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java @@ -27,12 +27,13 @@ public class TestDynamicCompileIT extends TestSuiteBase { + private final String basePath = "/dynamic_compile/conf/"; + @TestTemplate public void testDynamicSingleCompileGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/single_dynamic_groovy_compile_transform.conf"); + container.executeJob(basePath + "single_dynamic_groovy_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -40,7 +41,7 @@ public void testDynamicSingleCompileGroovy(TestContainer container) public void testDynamicSingleCompileJava(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob("/dynamic_compile/single_dynamic_java_compile_transform.conf"); + container.executeJob(basePath + "single_dynamic_java_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -48,8 +49,7 @@ public void testDynamicSingleCompileJava(TestContainer container) public void testDynamicMultipleCompileGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf"); + container.executeJob(basePath + "multiple_dynamic_groovy_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -57,8 +57,7 @@ public void testDynamicMultipleCompileGroovy(TestContainer container) public void testDynamicMultipleCompileJava(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/multiple_dynamic_java_compile_transform.conf"); + container.executeJob(basePath + "multiple_dynamic_java_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -66,8 +65,25 @@ public void testDynamicMultipleCompileJava(TestContainer container) public void testDynamicMixedCompileJavaAndGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf"); + container.executeJob(basePath + "mixed_dynamic_groovy_java_compile_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @TestTemplate + public void testDynamicSinglePathGroovy(TestContainer container) + throws IOException, InterruptedException { + container.copyFileToContainer("/dynamic_compile/source_file/GroovyFile", "/tmp/GroovyFile"); + Container.ExecResult execResult = + container.executeJob(basePath + "single_groovy_path_compile.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @TestTemplate + public void testDynamicSinglePathJava(TestContainer container) + throws IOException, InterruptedException { + container.copyFileToContainer("/dynamic_compile/source_file/JavaFile", "/tmp/JavaFile"); + Container.ExecResult execResult = + container.executeJob(basePath + "single_java_path_compile.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf index 5c32e8d5a03..e91765fbf3c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf @@ -43,6 +43,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -80,6 +81,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf index 31756b99415..8689404a17e 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf @@ -40,6 +40,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor @@ -73,6 +74,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf index 94e3a41272c..9e59a5e5350 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf @@ -43,6 +43,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -80,6 +81,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf index c478d33ddc5..7958b880765 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf @@ -40,6 +40,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf similarity index 99% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf index d3a735b6300..b65877d465c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf @@ -40,6 +40,7 @@ DynamicCompile { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf new file mode 100644 index 00000000000..c9b00bdee80 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf @@ -0,0 +1,85 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + } + } + } +} + +transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="GROOVY" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/GroovyFile""" + + } +} + +sink { + Assert { + source_table_name = "fake1" + rules = + { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 100 + } + ], + field_rules = [ + { + field_name = id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = aa + field_type = string + field_value = [ + { + rule_type = NOT_NULL + equals_to = "AA" + + } + + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf new file mode 100644 index 00000000000..3925dbe91e8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf @@ -0,0 +1,86 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + } + } + } +} + +transform { +DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="JAVA" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/JavaFile""" + + + } +} + +sink { + Assert { + source_table_name = "fake1" + rules = + { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 100 + } + ], + field_rules = [ + { + field_name = id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = col1 + field_type = string + field_value = [ + { + rule_type = NOT_NULL + equals_to = "test1" + + } + + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile new file mode 100644 index 00000000000..9bb6a8fcdfe --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile @@ -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. + */ +import org.apache.seatunnel.api.table.catalog.Column +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor +import org.apache.seatunnel.api.table.catalog.CatalogTable +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.type.*; +import java.util.ArrayList; +class demo { + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + List columns = new ArrayList<>(); + PhysicalColumn destColumn = + PhysicalColumn.of( + "aa", + BasicType.STRING_TYPE, + 10, + true, + "", + ""); + columns.add(destColumn); + return columns.toArray(new Column[0]); + } + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + Object[] fieldValues = new Object[1]; + fieldValues[0]="AA" + return fieldValues; + } +}; \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile new file mode 100644 index 00000000000..7d1947c077e --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; + +import java.util.ArrayList; + + + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + + ArrayList columns = new ArrayList(); + PhysicalColumn destColumn = + PhysicalColumn.of("col1", BasicType.STRING_TYPE, 10, true, "", ""); + return new Column[] {destColumn}; + } + + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + + Object[] fieldValues = new Object[1]; + fieldValues[0] = "test1"; + return fieldValues; + } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java new file mode 100644 index 00000000000..9b8c83a89df --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java @@ -0,0 +1,23 @@ +/* + * 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.transform.dynamiccompile; + +public enum CompilePattern { + SOURCE_CODE, + ABSOLUTE_PATH +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java index d798871401c..ea55569420d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.common.utils.FileUtils; import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.transform.common.MultipleFieldOutputTransform; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -28,6 +29,8 @@ import org.apache.seatunnel.transform.dynamiccompile.parse.JavaClassParse; import org.apache.seatunnel.transform.exception.TransformException; +import java.nio.file.Paths; + import static org.apache.seatunnel.transform.dynamiccompile.CompileTransformErrorCode.COMPILE_TRANSFORM_ERROR_CODE; public class DynamicCompileTransform extends MultipleFieldOutputTransform { @@ -39,6 +42,8 @@ public class DynamicCompileTransform extends MultipleFieldOutputTransform { private final String sourceCode; + private final CompilePattern compilePattern; + private AbstractParse DynamicCompileParse; public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { @@ -51,7 +56,18 @@ public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catal } else if (CompileLanguage.JAVA.equals(compileLanguage)) { DynamicCompileParse = new JavaClassParse(); } - sourceCode = readonlyConfig.get(DynamicCompileTransformConfig.SOURCE_CODE); + compilePattern = readonlyConfig.get(DynamicCompileTransformConfig.COMPILE_PATTERN); + + if (CompilePattern.SOURCE_CODE.equals(compilePattern)) { + sourceCode = readonlyConfig.get(DynamicCompileTransformConfig.SOURCE_CODE); + } else { + // NPE will never happen because it is required in the ABSOLUTE_PATH mode + sourceCode = + FileUtils.readFileToStr( + Paths.get( + readonlyConfig.get( + DynamicCompileTransformConfig.ABSOLUTE_PATH))); + } } @Override @@ -65,7 +81,7 @@ protected Column[] getOutputColumns() { try { result = ReflectionUtils.invoke( - DynamicCompileParse.parseClass(sourceCode).newInstance(), + getCompileLanguageInstance(), getInlineOutputColumns, inputCatalogTable); @@ -82,13 +98,17 @@ protected Object[] getOutputFieldValues(SeaTunnelRowAccessor inputRow) { try { result = ReflectionUtils.invoke( - DynamicCompileParse.parseClass(sourceCode).newInstance(), - getInlineOutputFieldValues, - inputRow); + getCompileLanguageInstance(), getInlineOutputFieldValues, inputRow); } catch (Exception e) { throw new TransformException(COMPILE_TRANSFORM_ERROR_CODE, e.getMessage()); } return (Object[]) result; } + + private Object getCompileLanguageInstance() + throws InstantiationException, IllegalAccessException { + Class compileClass = DynamicCompileParse.parseClassSourceCode(sourceCode); + return compileClass.newInstance(); + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java index 48a47d03830..f975ba28444 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java @@ -39,4 +39,16 @@ public class DynamicCompileTransformConfig implements Serializable { .enumType(CompileLanguage.class) .noDefaultValue() .withDescription("compile language"); + + public static final Option ABSOLUTE_PATH = + Options.key("absolute_path") + .stringType() + .noDefaultValue() + .withDescription("absolute_path"); + + public static final Option COMPILE_PATTERN = + Options.key("compile_pattern") + .enumType(CompilePattern.class) + .defaultValue(CompilePattern.SOURCE_CODE) + .withDescription("compile_pattern"); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java index 422bb0ff146..195102c4d91 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java @@ -38,7 +38,15 @@ public OptionRule optionRule() { return OptionRule.builder() .required( DynamicCompileTransformConfig.COMPILE_LANGUAGE, + DynamicCompileTransformConfig.COMPILE_PATTERN) + .conditional( + DynamicCompileTransformConfig.COMPILE_PATTERN, + CompilePattern.SOURCE_CODE, DynamicCompileTransformConfig.SOURCE_CODE) + .conditional( + DynamicCompileTransformConfig.COMPILE_PATTERN, + CompilePattern.ABSOLUTE_PATH, + DynamicCompileTransformConfig.ABSOLUTE_PATH) .build(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java index 906e9c26347..51d94fa1662 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java @@ -21,5 +21,5 @@ public abstract class AbstractParse implements Serializable { - public abstract Class parseClass(String sourceCode); + public abstract Class parseClassSourceCode(String sourceCode); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java similarity index 97% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java index c4afd47e25d..3d8d58fd82d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java @@ -20,7 +20,7 @@ import java.util.concurrent.ConcurrentHashMap; -public abstract class ParseUtil { +public abstract class AbstractParser { protected static ConcurrentHashMap> classCache = new ConcurrentHashMap<>(); // Abstraction layer: Do not want to serialize and pass the classloader protected static String getClassKey(String sourceCode) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java index d94607eb1f5..7ae95da6288 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java @@ -20,7 +20,7 @@ public class GroovyClassParse extends AbstractParse { @Override - public Class parseClass(String sourceCode) { - return GroovyClassUtil.parseWithCache(sourceCode); + public Class parseClassSourceCode(String sourceCode) { + return GroovyClassParser.parseSourceCodeWithCache(sourceCode); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java similarity index 89% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java index 5fab0e8761f..c951335e371 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java @@ -18,10 +18,10 @@ import groovy.lang.GroovyClassLoader; -public class GroovyClassUtil extends ParseUtil { +public class GroovyClassParser extends AbstractParser { private static final GroovyClassLoader groovyClassLoader = new GroovyClassLoader(); - public static Class parseWithCache(String sourceCode) { + public static Class parseSourceCodeWithCache(String sourceCode) { return classCache.computeIfAbsent( getClassKey(sourceCode), clazz -> groovyClassLoader.parseClass(sourceCode)); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java index 3cd5bdd96e9..9b77963eea6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java @@ -19,7 +19,7 @@ public class JavaClassParse extends AbstractParse { @Override - public Class parseClass(String sourceCode) { - return JavaClassUtil.parseWithCache(sourceCode); + public Class parseClassSourceCode(String sourceCode) { + return JavaClassParser.parseSourceCodeWithCache(sourceCode); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java similarity index 72% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java index 344b2708d4a..d9bee066f72 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java @@ -21,24 +21,29 @@ import java.util.function.Function; -public class JavaClassUtil extends ParseUtil { - - public static Class parseWithCache(String sourceCode) { +public class JavaClassParser extends AbstractParser { + public static Class parseSourceCodeWithCache(String sourceCode) { return classCache.computeIfAbsent( getClassKey(sourceCode), new Function>() { @Override public Class apply(String classKey) { - try { - ClassBodyEvaluator cbe = new ClassBodyEvaluator(); - cbe.cook(sourceCode); - return cbe.getClazz(); - - } catch (CompileException e) { - throw new RuntimeException(e); - } + return getInnerClass(sourceCode); } }); } + + private static Class getInnerClass(String FilePathOrSourceCode) { + try { + ClassBodyEvaluator cbe = new ClassBodyEvaluator(); + + cbe.cook(FilePathOrSourceCode); + + return cbe.getClazz(); + + } catch (CompileException e) { + throw new RuntimeException(e); + } + } } From 34a6b8e9f62d9e04da554062b46949c7075176ed Mon Sep 17 00:00:00 2001 From: Jast Date: Wed, 31 Jul 2024 14:43:11 +0800 Subject: [PATCH 13/32] [hotfix][connector-v2-hbase]fix and optimize hbase source problem (#7148) * [hotfix][improve][doc]optimize connector hbase source * [doc]add dependent document * [doc]update dependent document * [improve]improve static use * [hotfix]add test case * [hotfix]add test case --------- Co-authored-by: Jia Fan --- docs/en/connector-v2/source/Hbase.md | 109 ++++++++------- docs/zh/connector-v2/source/Hbase.md | 96 +++++++++++++ docs/zh/connector-v2/source/common-options.md | 81 +++++++++++ .../seatunnel/hbase/config/HbaseConfig.java | 27 +++- .../hbase/config/HbaseParameters.java | 24 +++- .../seatunnel/hbase/sink/HbaseSink.java | 2 +- .../seatunnel/hbase/source/HbaseSource.java | 6 +- .../hbase/source/HbaseSourceFactory.java | 1 - .../hbase/source/HbaseSourceReader.java | 30 ++-- .../e2e/connector/hbase/HbaseIT.java | 37 +++-- ...ase-source-to-assert-with-batch-query.conf | 132 ++++++++++++++++++ 11 files changed, 455 insertions(+), 90 deletions(-) create mode 100644 docs/zh/connector-v2/source/Hbase.md create mode 100644 docs/zh/connector-v2/source/common-options.md create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf diff --git a/docs/en/connector-v2/source/Hbase.md b/docs/en/connector-v2/source/Hbase.md index 677b827fb29..753d68eb6e8 100644 --- a/docs/en/connector-v2/source/Hbase.md +++ b/docs/en/connector-v2/source/Hbase.md @@ -1,12 +1,12 @@ # Hbase -> Hbase source connector +> Hbase Source Connector ## Description -Read data from Apache Hbase. +Reads data from Apache Hbase. -## Key features +## Key Features - [x] [batch](../../concept/connector-v2-features.md) - [ ] [stream](../../concept/connector-v2-features.md) @@ -17,75 +17,80 @@ Read data from Apache Hbase. ## Options -| name | type | required | default value | -|--------------------|--------|----------|---------------| -| zookeeper_quorum | string | yes | - | -| table | string | yes | - | -| query_columns | list | yes | - | -| schema | config | yes | - | -| hbase_extra_config | string | no | - | -| common-options | | no | - | +| Name | Type | Required | Default | +|--------------------|---------|----------|---------| +| zookeeper_quorum | string | Yes | - | +| table | string | Yes | - | +| schema | config | Yes | - | +| hbase_extra_config | string | No | - | +| caching | int | No | -1 | +| batch | int | No | -1 | +| cache_blocks | boolean | No | false | +| common-options | | No | - | ### zookeeper_quorum [string] -The zookeeper cluster host of hbase, example: "hadoop001:2181,hadoop002:2181,hadoop003:2181" +The zookeeper quorum for Hbase cluster hosts, e.g., "hadoop001:2181,hadoop002:2181,hadoop003:2181". ### table [string] -The table name you want to write, example: "seatunnel" - -### query_columns [list] - -The column name which you want to query in the table. If you want to query the rowkey column, please set "rowkey" in query_columns. -Other column format should be: columnFamily:columnName, example: ["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] +The name of the table to write to, e.g., "seatunnel". ### schema [config] -Hbase uses byte arrays for storage. Therefore, you need to configure data types for each column in a table. For more information, see: [guide](../../concept/schema-feature.md#how-to-declare-type-supported). +Hbase stores data in byte arrays. Therefore, you need to configure the data types for each column in the table. For more information, see: [guide](../../concept/schema-feature.md#how-to-declare-type-supported). ### hbase_extra_config [config] -The extra configuration of hbase +Additional configurations for Hbase. + +### caching + +The caching parameter sets the number of rows fetched per server trip during scans. This reduces round-trips between client and server, improving scan efficiency. Default: -1. + +### batch + +The batch parameter sets the maximum number of columns returned per scan. This is useful for rows with many columns to avoid fetching excessive data at once, thus saving memory and improving performance. -### common options +### cache_blocks -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details +The cache_blocks parameter determines whether to cache data blocks during scans. By default, HBase caches data blocks during scans. Setting this to false reduces memory usage during scans. Default in SeaTunnel: false. -## Examples +### common-options + +Common parameters for Source plugins, refer to [Common Source Options](common-options.md). + +## Example ```bash source { Hbase { - zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" - table = "seatunnel_test" - query_columns=["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] - schema = { - columns = [ - { - name = rowkey - type = string - }, - { - name = "columnFamily1:column1" - type = boolean - }, - { - name = "columnFamily1:column1" - type = double - }, - { - name = "columnFamily2:column1" - type = bigint - } - ] - } + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = "rowkey" + type = string + }, + { + name = "columnFamily1:column1" + type = boolean + }, + { + name = "columnFamily1:column2" + type = double + }, + { + name = "columnFamily2:column1" + type = bigint + } + ] + } } } ``` -## Changelog - -### next version - -- Add Hbase Source Connector - diff --git a/docs/zh/connector-v2/source/Hbase.md b/docs/zh/connector-v2/source/Hbase.md new file mode 100644 index 00000000000..5f15a30b99a --- /dev/null +++ b/docs/zh/connector-v2/source/Hbase.md @@ -0,0 +1,96 @@ +# Hbase + +> Hbase 源连接器 + +## 描述 + +从 Apache Hbase 读取数据。 + +## 主要功能 + +- [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) + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|--------------------|---------|----|-------| +| zookeeper_quorum | string | 是 | - | +| table | string | 是 | - | +| schema | config | 是 | - | +| hbase_extra_config | string | 否 | - | +| caching | int | 否 | -1 | +| batch | int | 否 | -1 | +| cache_blocks | boolean | 否 | false | +| common-options | | 否 | - | + +### zookeeper_quorum [string] + +hbase的zookeeper集群主机,例如:“hadoop001:2181,hadoop002:2181,hadoop003:2181” + +### table [string] + +要写入的表名,例如:“seatunnel” + +### schema [config] + +Hbase 使用字节数组进行存储。因此,您需要为表中的每一列配置数据类型。有关更多信息,请参阅:[guide](../../concept/schema-feature.md#how-to-declare-type-supported)。 + +### hbase_extra_config [config] + +hbase 的额外配置 + +### caching + +caching 参数用于设置在扫描过程中一次从服务器端获取的行数。这可以减少客户端与服务器之间的往返次数,从而提高扫描效率。默认值:-1 + +### batch + +batch 参数用于设置在扫描过程中每次返回的最大列数。这对于处理有很多列的行特别有用,可以避免一次性返回过多数据,从而节省内存并提高性能。 + +### cache_blocks + +cache_blocks 参数用于设置在扫描过程中是否缓存数据块。默认情况下,HBase 会在扫描时将数据块缓存到块缓存中。如果设置为 false,则在扫描过程中不会缓存数据块,从而减少内存的使用。在SeaTunnel中默认值为: false + +### 常用选项 + +Source 插件常用参数,具体请参考 [Source 常用选项](common-options.md) + +## 示例 + +```bash +source { + Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = "rowkey" + type = string + }, + { + name = "columnFamily1:column1" + type = boolean + }, + { + name = "columnFamily1:column2" + type = double + }, + { + name = "columnFamily2:column1" + type = bigint + } + ] + } + } +} +``` + diff --git a/docs/zh/connector-v2/source/common-options.md b/docs/zh/connector-v2/source/common-options.md new file mode 100644 index 00000000000..902dca2c195 --- /dev/null +++ b/docs/zh/connector-v2/source/common-options.md @@ -0,0 +1,81 @@ +# Source Common Options + +> Source connector 的常用参数 + +| 名称 | 类型 | 必填 | 默认值 | 描述 | +|-------------------|--------|----|-----|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| result_table_name | String | 否 | - | 当未指定 `result_table_name` 时,此插件处理的数据将不会被注册为可由其他插件直接访问的数据集 `(dataStream/dataset)`,或称为临时表 `(table)`。
当指定了 `result_table_name` 时,此插件处理的数据将被注册为可由其他插件直接访问的数据集 `(dataStream/dataset)`,或称为临时表 `(table)`。此处注册的数据集 `(dataStream/dataset)` 可通过指定 `source_table_name` 直接被其他插件访问。 | +| parallelism | Int | 否 | - | 当未指定 `parallelism` 时,默认使用环境中的 `parallelism`。
当指定了 `parallelism` 时,将覆盖环境中的 `parallelism` 设置。 | + +# 重要提示 + +在作业配置中使用 `result_table_name` 时,必须设置 `source_table_name` 参数。 + +## 任务示例 + +### 简单示例 + +> 注册一个流或批处理数据源,并在注册时返回表名 `fake_table` + +```bash +source { + FakeSourceStream { + result_table_name = "fake_table" + } +} +``` + +### 复杂示例 + +> 这是将Fake数据源转换并写入到两个不同的目标中 + +```bash +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + age = "int" + c_timestamp = "timestamp" + c_date = "date" + c_map = "map" + c_array = "array" + c_decimal = "decimal(30, 8)" + c_row = { + c_row = { + c_int = int + } + } + } + } + } +} + +transform { + Sql { + source_table_name = "fake" + result_table_name = "fake1" + # 查询表名必须与字段 'source_table_name' 相同 + query = "select id, regexp_replace(name, '.+', 'b') as name, age+1 as age, pi() as pi, c_timestamp, c_date, c_map, c_array, c_decimal, c_row from fake" + } + # SQL 转换支持基本函数和条件操作 + # 但不支持复杂的 SQL 操作,包括:多源表/行 JOIN 和聚合操作等 +} + +sink { + Console { + source_table_name = "fake1" + } + Console { + source_table_name = "fake" + } +} +``` + diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java index 88c068bee11..44a5640ffed 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java @@ -42,12 +42,6 @@ public class HbaseConfig { .noDefaultValue() .withDescription("Hbase rowkey column"); - public static final Option> QUERY_COLUMNS = - Options.key("query_columns") - .listType() - .noDefaultValue() - .withDescription("query Hbase columns"); - public static final Option ROWKEY_DELIMITER = Options.key("rowkey_delimiter") .stringType() @@ -104,6 +98,27 @@ public class HbaseConfig { .withDescription( "The expiration time configuration for writing hbase data. The default value is -1, indicating no expiration time."); + public static final Option HBASE_CACHE_BLOCKS_CONFIG = + Options.key("cache_blocks") + .booleanType() + .defaultValue(false) + .withDescription( + "When it is false, data blocks are not cached. When it is true, data blocks are cached. This value should be set to false when scanning a large amount of data to reduce memory consumption. The default value is false"); + + public static final Option HBASE_CACHING_CONFIG = + Options.key("caching") + .intType() + .defaultValue(-1) + .withDescription( + "Set the number of rows read from the server each time can reduce the number of round trips between the client and the server, thereby improving performance. The default value is -1."); + + public static final Option HBASE_BATCH_CONFIG = + Options.key("batch") + .intType() + .defaultValue(-1) + .withDescription( + "Set the batch size to control the maximum number of cells returned each time, thereby controlling the amount of data returned by a single RPC call. The default value is -1."); + public enum NullMode { SKIP, EMPTY; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java index 490e2481070..c25f04b3753 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java @@ -30,10 +30,12 @@ import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ENCODING; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_BATCH_CONFIG; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_CACHE_BLOCKS_CONFIG; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_CACHING_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_EXTRA_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_TTL_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.NULL_MODE; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_DELIMITER; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; @@ -60,8 +62,14 @@ public class HbaseParameters implements Serializable { private Map hbaseExtraConfig; + @Builder.Default private int caching = HBASE_CACHING_CONFIG.defaultValue(); + + @Builder.Default private int batch = HBASE_BATCH_CONFIG.defaultValue(); + @Builder.Default private Long ttl = HBASE_TTL_CONFIG.defaultValue(); + @Builder.Default private boolean cacheBlocks = HBASE_CACHE_BLOCKS_CONFIG.defaultValue(); + @Builder.Default private String rowkeyDelimiter = ROWKEY_DELIMITER.defaultValue(); @Builder.Default private HbaseConfig.NullMode nullMode = NULL_MODE.defaultValue(); @@ -72,7 +80,7 @@ public class HbaseParameters implements Serializable { @Builder.Default private HbaseConfig.EnCoding enCoding = ENCODING.defaultValue(); - public static HbaseParameters buildWithConfig(Config pluginConfig) { + public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); // required parameters @@ -113,18 +121,26 @@ public static HbaseParameters buildWithConfig(Config pluginConfig) { return builder.build(); } - public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { + public static HbaseParameters buildWithSourceConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); // required parameters builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); builder.table(pluginConfig.getString(TABLE.key())); - builder.columns(pluginConfig.getStringList(QUERY_COLUMNS.key())); if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig)); } + if (pluginConfig.hasPath(HBASE_CACHING_CONFIG.key())) { + builder.caching(pluginConfig.getInt(HBASE_CACHING_CONFIG.key())); + } + if (pluginConfig.hasPath(HBASE_BATCH_CONFIG.key())) { + builder.batch(pluginConfig.getInt(HBASE_BATCH_CONFIG.key())); + } + if (pluginConfig.hasPath(HBASE_CACHE_BLOCKS_CONFIG.key())) { + builder.cacheBlocks(pluginConfig.getBoolean(HBASE_CACHE_BLOCKS_CONFIG.key())); + } return builder.build(); } } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java index 848e1e82053..4f7b929223f 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java @@ -79,7 +79,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SINK, result.getMsg())); } - this.hbaseParameters = HbaseParameters.buildWithConfig(pluginConfig); + this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); if (hbaseParameters.getFamilyNames().size() == 0) { throw new HbaseConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java index 869e33f6235..3aca3161516 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java @@ -44,7 +44,6 @@ import java.util.List; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ZOOKEEPER_QUORUM; @@ -68,8 +67,7 @@ public String getPluginName() { HbaseSource(Config pluginConfig) { this.pluginConfig = pluginConfig; CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key(), QUERY_COLUMNS.key()); + CheckConfigUtil.checkAllExists(pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key()); if (!result.isSuccess()) { throw new HbaseConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, @@ -77,7 +75,7 @@ public String getPluginName() { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SOURCE, result.getMsg())); } - this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); + this.hbaseParameters = HbaseParameters.buildWithSourceConfig(pluginConfig); this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java index 4eec3e00482..2de385dbd18 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java @@ -45,7 +45,6 @@ public OptionRule optionRule() { return OptionRule.builder() .required(HbaseConfig.ZOOKEEPER_QUORUM) .required(HbaseConfig.TABLE) - .required(HbaseConfig.QUERY_COLUMNS) .build(); } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java index 556374844e9..526ac826db1 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; @@ -39,13 +40,13 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Deque; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.stream.Collectors; @Slf4j public class HbaseSourceReader implements SourceReader { @@ -54,7 +55,6 @@ public class HbaseSourceReader implements SourceReader namesMap; - private final Set columnFamilies = new LinkedHashSet<>(); private final SourceReader.Context context; private final SeaTunnelRowType seaTunnelRowType; private volatile boolean noMoreSplit = false; @@ -74,16 +74,17 @@ public HbaseSourceReader( this.seaTunnelRowType = seaTunnelRowType; this.namesMap = Maps.newConcurrentMap(); - this.columnNames = hbaseParameters.getColumns(); + this.columnNames = + Arrays.asList(seaTunnelRowType.getFieldNames()).stream() + .filter(name -> !ROW_KEY.equals(name)) + .collect(Collectors.toList()); // Check if input column names are in format: [ columnFamily:column ]. this.columnNames.stream() - .peek( + .forEach( column -> Preconditions.checkArgument( - (column.contains(":") && column.split(":").length == 2) - || this.ROW_KEY.equalsIgnoreCase(column), - "Invalid column names, it should be [ColumnFamily:Column] format")) - .forEach(column -> this.columnFamilies.add(column.split(":")[0])); + column.contains(":") && column.split(":").length == 2, + "Invalid column names, it should be [ColumnFamily:Column] format")); connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); } @@ -122,6 +123,15 @@ public void pollNext(Collector output) throws Exception { Scan scan = new Scan(); scan.withStartRow(split.getStartRow(), true); scan.withStopRow(split.getEndRow(), true); + scan.setCacheBlocks(hbaseParameters.isCacheBlocks()); + scan.setCaching(hbaseParameters.getCaching()); + scan.setBatch(hbaseParameters.getBatch()); + for (String columnName : this.columnNames) { + String[] columnNameSplit = columnName.split(":"); + scan.addColumn( + Bytes.toBytes(columnNameSplit[0]), + Bytes.toBytes(columnNameSplit[1])); + } this.currentScanner = this.connection .getTable(TableName.valueOf(hbaseParameters.getTable())) @@ -152,7 +162,7 @@ private byte[][] convertRawRow(Result result) { byte[] bytes; try { // handle rowkey column - if (this.ROW_KEY.equals(columnName)) { + if (ROW_KEY.equals(columnName)) { bytes = result.getRow(); } else { byte[][] arr = this.namesMap.get(columnName); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java index 13a7a8805a6..85ceef92353 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java @@ -93,18 +93,7 @@ public void tearDown() throws Exception { @TestTemplate public void testHbaseSink(TestContainer container) throws IOException, InterruptedException { - deleteData(table); - Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf"); - Assertions.assertEquals(0, sinkExecResult.getExitCode()); - Table hbaseTable = hbaseConnection.getTable(table); - Scan scan = new Scan(); - ResultScanner scanner = hbaseTable.getScanner(scan); - ArrayList results = new ArrayList<>(); - for (Result result : scanner) { - results.add(result); - } - Assertions.assertEquals(results.size(), 5); - scanner.close(); + fakeToHbase(container); Container.ExecResult sourceExecResult = container.executeJob("/hbase-to-assert.conf"); Assertions.assertEquals(0, sourceExecResult.getExitCode()); } @@ -177,6 +166,30 @@ public void testHbaseSinkAssignCfSink(TestContainer container) Assertions.assertEquals(cf2Count, 5); } + @TestTemplate + public void testHbaseSourceWithBatchQuery(TestContainer container) + throws IOException, InterruptedException { + fakeToHbase(container); + Container.ExecResult sourceExecResult = + container.executeJob("/hbase-source-to-assert-with-batch-query.conf"); + Assertions.assertEquals(0, sourceExecResult.getExitCode()); + } + + private void fakeToHbase(TestContainer container) throws IOException, InterruptedException { + deleteData(table); + Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf"); + Assertions.assertEquals(0, sinkExecResult.getExitCode()); + Table hbaseTable = hbaseConnection.getTable(table); + Scan scan = new Scan(); + ResultScanner scanner = hbaseTable.getScanner(scan); + ArrayList results = new ArrayList<>(); + for (Result result : scanner) { + results.add(result); + } + Assertions.assertEquals(results.size(), 5); + scanner.close(); + } + private void deleteData(TableName table) throws IOException { Table hbaseTable = hbaseConnection.getTable(table); Scan scan = new Scan(); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf new file mode 100644 index 00000000000..c89cf28e25d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf @@ -0,0 +1,132 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test" + query_columns=["rowkey", "info:age", "info:c_double", "info:c_boolean","info:c_bigint","info:c_smallint","info:c_tinyint","info:c_float"] + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = rowkey + type = string + }, + { + name = "info:age" + type = int + }, + { + name = "info:c_double" + type = double + }, + { + name = "info:c_boolean" + type = boolean + }, + { + name = "info:c_bigint" + type = bigint + }, + { + name = "info:c_smallint" + type = smallint + }, + { + name = "info:c_tinyint" + type = tinyint + }, + { + name = "info:c_float" + type = float + } + ] + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + }, + { + rule_type = MIN_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = rowkey + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_boolean" + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_double" + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_bigint" + field_type = bigint + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:age" + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file From 82f5d8c71212b0ceb4a57d9cc7cd97cab5aa6d4e Mon Sep 17 00:00:00 2001 From: hailin0 Date: Sat, 3 Aug 2024 12:02:19 +0800 Subject: [PATCH 14/32] [Improve][SQL-Transform] Remove escape identifier from output fields (#7297) --- .../resources/sql_transform/func_system.conf | 9 +- .../transform/sql/zeta/ZetaSQLEngine.java | 10 +- .../transform/sql/zeta/ZetaSQLFunction.java | 26 +++- .../transform/sql/zeta/ZetaSQLType.java | 16 ++- .../transform/sql/SQLTransformTest.java | 136 ++++++++++++++++++ 5 files changed, 191 insertions(+), 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index 14f41665e34..a189c7c2ddc 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7 from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7, name as `apply` from fake" } } @@ -164,6 +164,13 @@ sink { rule_type = NOT_NULL } ] + }, + { + field_name = "apply" + field_type = "string" + field_value = [ + {equals_to = "Joy Ding"} + ] } ] } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index 42f5d8205dc..993b4e0a3c2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -50,6 +50,8 @@ public class ZetaSQLEngine implements SQLEngine { private static final Logger log = LoggerFactory.getLogger(ZetaSQLEngine.class); + public static final String ESCAPE_IDENTIFIER = "`"; + private String inputTableName; @Nullable private String catalogTableName; private SeaTunnelRowType inputRowType; @@ -193,9 +195,13 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { } else if (selectItem instanceof SelectExpressionItem) { SelectExpressionItem expressionItem = (SelectExpressionItem) selectItem; Expression expression = expressionItem.getExpression(); - if (expressionItem.getAlias() != null) { - fieldNames[idx] = expressionItem.getAlias().getName(); + String aliasName = expressionItem.getAlias().getName(); + if (aliasName.startsWith(ESCAPE_IDENTIFIER) + && aliasName.endsWith(ESCAPE_IDENTIFIER)) { + aliasName = aliasName.substring(1, aliasName.length() - 1); + } + fieldNames[idx] = aliasName; } else { if (expression instanceof Column) { fieldNames[idx] = ((Column) expression).getColumnName(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 44b9ca20b7c..a6221e4a277 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -227,6 +227,13 @@ public Object computeForValue(Expression expression, Object[] inputFields) { Column columnExp = (Column) expression; String columnName = columnExp.getColumnName(); int index = inputRowType.indexOf(columnName, false); + if (index == -1 + && columnName.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && columnName.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + columnName = columnName.substring(1, columnName.length() - 1); + index = inputRowType.indexOf(columnName, false); + } + if (index != -1) { return inputFields[index]; } else { @@ -237,11 +244,26 @@ public Object computeForValue(Expression expression, Object[] inputFields) { SeaTunnelRow parRowValues = new SeaTunnelRow(inputFields); Object res = parRowValues; for (int i = 0; i < deep; i++) { + String key = columnNames[i]; if (parDataType instanceof MapType) { - return ((Map) res).get(columnNames[i]); + Map mapValue = ((Map) res); + if (mapValue.containsKey(key)) { + return mapValue.get(key); + } else if (key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + return mapValue.get(key); + } + return null; } parRowValues = (SeaTunnelRow) res; - int idx = ((SeaTunnelRowType) parDataType).indexOf(columnNames[i], false); + int idx = ((SeaTunnelRowType) parDataType).indexOf(key, false); + if (idx == -1 + && key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + idx = ((SeaTunnelRowType) parDataType).indexOf(key, false); + } if (idx == -1) { throw new IllegalArgumentException( String.format("can't find field [%s]", fullyQualifiedName)); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java index 45b269bae67..9b527ae8c2f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java @@ -111,6 +111,13 @@ public SeaTunnelDataType getExpressionType(Expression expression) { Column columnExp = (Column) expression; String columnName = columnExp.getColumnName(); int index = inputRowType.indexOf(columnName, false); + if (index == -1 + && columnName.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && columnName.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + columnName = columnName.substring(1, columnName.length() - 1); + index = inputRowType.indexOf(columnName, false); + } + if (index != -1) { return inputRowType.getFieldType(index); } else { @@ -121,7 +128,14 @@ public SeaTunnelDataType getExpressionType(Expression expression) { SeaTunnelRowType parRowType = inputRowType; SeaTunnelDataType filedTypeRes = null; for (int i = 0; i < deep; i++) { - int idx = parRowType.indexOf(columnNames[i], false); + String key = columnNames[i]; + int idx = parRowType.indexOf(key, false); + if (idx == -1 + && key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + idx = parRowType.indexOf(key, false); + } if (idx == -1) { throw new IllegalArgumentException( String.format("can't find field [%s]", fullyQualifiedName)); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index 854fae5cb32..ff253eac210 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -19,18 +19,22 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; 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.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Objects; @@ -144,4 +148,136 @@ private CatalogTable getCatalogTable() { new ArrayList<>(), "It has column information."); } + + @Test + public void testEscapeIdentifier() { + String tableName = "test"; + String[] fields = new String[] {"id", "apply"}; + CatalogTable table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, BasicType.STRING_TYPE + })); + ReadonlyConfig config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, trim(`apply`) as `apply` from test where `apply` = 'a'")); + SQLTransform sqlTransform = new SQLTransform(config, table); + TableSchema tableSchema = sqlTransform.transformTableSchema(); + SeaTunnelRow result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals("a", result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("b")})); + Assertions.assertNull(result); + + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, IFNULL(`apply`, '1') as `apply` from test where `apply` = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.LONG_TYPE})); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, `apply` + 1 as `apply` from test where `apply` > 0")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(1)})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals(BasicType.LONG_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals(Long.valueOf(2), result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(0)})); + Assertions.assertNull(result); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + })); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, `apply`.k1 as `apply` from test where `apply`.k1 = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("k1", "a") + })); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("k1", "b") + })); + Assertions.assertNull(result); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + new String[] {"id", "map"}, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + })); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, map.`apply` as `apply` from test where map.`apply` = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("apply", "a") + })); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + } } From 2fd4eec22aeb3a35c558eeecbc35d35ad217db10 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 5 Aug 2024 11:35:02 +0800 Subject: [PATCH 15/32] [Fix][Doc] Fix hybrid cluster deployment document display error (#7306) --- docs/en/seatunnel-engine/hybrid-cluster-deployment.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md index c969376f162..60260f91bb0 100644 --- a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md @@ -178,10 +178,6 @@ hazelcast: TCP is the recommended method for use in a standalone SeaTunnel Engine cluster. Alternatively, Hazelcast provides several other service discovery methods. For more details, please refer to [Hazelcast Network](https://docs.hazelcast.com/imdg/4.1/clusters/setting-up-clusters) --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - -sidebar_position: 5 -------------------- ### 5.3 IMap Persistence Configuration From 4f120ff34b1b5cf0536ff88cd9b571eff3b1fdcb Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 5 Aug 2024 14:34:01 +0800 Subject: [PATCH 16/32] [Improve] Update snapshot version to 2.3.7 (#7305) --- bin/install-plugin.cmd | 4 +-- bin/install-plugin.sh | 4 +-- .../en/seatunnel-engine/download-seatunnel.md | 6 ++-- docs/en/start-v2/kubernetes/kubernetes.mdx | 36 +++++++++---------- docs/en/start-v2/locally/deployment.md | 6 ++-- .../zh/seatunnel-engine/download-seatunnel.md | 8 ++--- docs/zh/start-v2/locally/deployment.md | 6 ++-- pom.xml | 2 +- tools/dependencies/known-dependencies.txt | 8 ++--- 9 files changed, 40 insertions(+), 40 deletions(-) diff --git a/bin/install-plugin.cmd b/bin/install-plugin.cmd index e4d7e27432b..799f1d2fd66 100644 --- a/bin/install-plugin.cmd +++ b/bin/install-plugin.cmd @@ -22,8 +22,8 @@ REM Get seatunnel home set "SEATUNNEL_HOME=%~dp0..\" echo Set SEATUNNEL_HOME to [%SEATUNNEL_HOME%] -REM Connector default version is 2.3.6, you can also choose a custom version. eg: 2.1.2: install-plugin.bat 2.1.2 -set "version=2.3.6" +REM Connector default version is 2.3.7, you can also choose a custom version. eg: 2.1.2: install-plugin.bat 2.1.2 +set "version=2.3.7" if not "%~1"=="" set "version=%~1" REM Create the lib directory diff --git a/bin/install-plugin.sh b/bin/install-plugin.sh index 2766112add6..43d0bcb837a 100755 --- a/bin/install-plugin.sh +++ b/bin/install-plugin.sh @@ -23,8 +23,8 @@ # get seatunnel home SEATUNNEL_HOME=$(cd $(dirname $0);cd ../;pwd) -# connector default version is 2.3.6, you can also choose a custom version. eg: 2.1.2: sh install-plugin.sh 2.1.2 -version=2.3.6 +# connector default version is 2.3.7, you can also choose a custom version. eg: 2.1.2: sh install-plugin.sh 2.1.2 +version=2.3.7 if [ -n "$1" ]; then version="$1" diff --git a/docs/en/seatunnel-engine/download-seatunnel.md b/docs/en/seatunnel-engine/download-seatunnel.md index ffbf833820a..e1ddd88b681 100644 --- a/docs/en/seatunnel-engine/download-seatunnel.md +++ b/docs/en/seatunnel-engine/download-seatunnel.md @@ -21,7 +21,7 @@ Go to the [Seatunnel Download Page](https://seatunnel.apache.org/download) to do Or you can also download it through the terminal. ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ Starting from the 2.2.0-beta version, the binary package no longer provides the sh bin/install-plugin.sh ``` -If you need a specific connector version, taking 2.3.6 as an example, you need to execute the following command. +If you need a specific connector version, taking 2.3.7 as an example, you need to execute the following command. ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` Usually you don't need all the connector plugins, so you can specify the plugins you need through configuring `config/plugin_config`, for example, if you only need the `connector-console` plugin, then you can modify the plugin.properties configuration file as follows. diff --git a/docs/en/start-v2/kubernetes/kubernetes.mdx b/docs/en/start-v2/kubernetes/kubernetes.mdx index b40e561ec72..7c5a4ac2795 100644 --- a/docs/en/start-v2/kubernetes/kubernetes.mdx +++ b/docs/en/start-v2/kubernetes/kubernetes.mdx @@ -44,7 +44,7 @@ To run the image with SeaTunnel, first create a `Dockerfile`: ```Dockerfile FROM flink:1.13 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -56,13 +56,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6-flink-1.13 -f Dockerfile . +docker build -t seatunnel:2.3.7-flink-1.13 -f Dockerfile . ``` -Image `seatunnel:2.3.6-flink-1.13` needs to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7-flink-1.13` needs to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6-flink-1.13 +minikube image load seatunnel:2.3.7-flink-1.13 ``` @@ -72,7 +72,7 @@ minikube image load seatunnel:2.3.6-flink-1.13 ```Dockerfile FROM openjdk:8 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -84,13 +84,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6 -f Dockerfile . +docker build -t seatunnel:2.3.7 -f Dockerfile . ``` -Image `seatunnel:2.3.6` need to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7` need to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6 +minikube image load seatunnel:2.3.7 ``` @@ -100,7 +100,7 @@ minikube image load seatunnel:2.3.6 ```Dockerfile FROM openjdk:8 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -112,13 +112,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6 -f Dockerfile . +docker build -t seatunnel:2.3.7 -f Dockerfile . ``` -Image `seatunnel:2.3.6` needs to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7` needs to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6 +minikube image load seatunnel:2.3.7 ``` @@ -191,7 +191,7 @@ none ]}> -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -245,7 +245,7 @@ kind: FlinkDeployment metadata: name: seatunnel-flink-streaming-example spec: - image: seatunnel:2.3.6-flink-1.13 + image: seatunnel:2.3.7-flink-1.13 flinkVersion: v1_13 flinkConfiguration: taskmanager.numberOfTaskSlots: "2" @@ -291,7 +291,7 @@ kubectl apply -f seatunnel-flink.yaml -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -334,7 +334,7 @@ metadata: spec: containers: - name: seatunnel - image: seatunnel:2.3.6 + image: seatunnel:2.3.7 command: ["/bin/sh","-c","/opt/seatunnel/bin/seatunnel.sh --config /data/seatunnel.streaming.conf -e local"] resources: limits: @@ -366,7 +366,7 @@ kubectl apply -f seatunnel.yaml -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -524,7 +524,7 @@ spec: spec: containers: - name: seatunnel - image: seatunnel:2.3.6 + image: seatunnel:2.3.7 imagePullPolicy: IfNotPresent ports: - containerPort: 5801 diff --git a/docs/en/start-v2/locally/deployment.md b/docs/en/start-v2/locally/deployment.md index 69cf5164e95..0d5f0e26d11 100644 --- a/docs/en/start-v2/locally/deployment.md +++ b/docs/en/start-v2/locally/deployment.md @@ -21,7 +21,7 @@ Visit the [SeaTunnel Download Page](https://seatunnel.apache.org/download) to do Or you can also download it through the terminal: ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ Starting from the 2.2.0-beta version, the binary package no longer provides the sh bin/install-plugin.sh ``` -If you need a specific connector version, taking 2.3.6 as an example, you need to execute the following command: +If you need a specific connector version, taking 2.3.7 as an example, you need to execute the following command: ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` Usually you don't need all connector plugins, so you can specify the plugins you need through configuring `config/plugin_config`. For example, if you only need the `connector-console` plugin, you can modify the plugin.properties configuration file as follows: diff --git a/docs/zh/seatunnel-engine/download-seatunnel.md b/docs/zh/seatunnel-engine/download-seatunnel.md index c108f4812a3..74281d0648f 100644 --- a/docs/zh/seatunnel-engine/download-seatunnel.md +++ b/docs/zh/seatunnel-engine/download-seatunnel.md @@ -21,7 +21,7 @@ import TabItem from '@theme/TabItem'; 或者您也可以通过终端下载 ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -31,13 +31,13 @@ tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" 从2.2.0-beta版本开始,二进制包不再默认提供连接器依赖,因此在第一次使用它时,您需要执行以下命令来安装连接器:(当然,您也可以从 [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) 手动下载连接器,然后将其移动至`connectors/seatunnel`目录下)。 ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` -如果您需要指定的连接器版本,以2.3.6为例,您需要执行如下命令 +如果您需要指定的连接器版本,以2.3.7为例,您需要执行如下命令 ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` 通常您并不需要所有的连接器插件,所以您可以通过配置`config/plugin_config`来指定您所需要的插件,例如,您只需要`connector-console`插件,那么您可以修改plugin.properties配置文件如下 diff --git a/docs/zh/start-v2/locally/deployment.md b/docs/zh/start-v2/locally/deployment.md index 9fa70f16040..167abeaeaab 100644 --- a/docs/zh/start-v2/locally/deployment.md +++ b/docs/zh/start-v2/locally/deployment.md @@ -21,7 +21,7 @@ import TabItem from '@theme/TabItem'; 或者您也可以通过终端下载: ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" sh bin/install-plugin.sh ``` -如果您需要指定的连接器版本,以2.3.6为例,您需要执行如下命令: +如果您需要指定的连接器版本,以2.3.7为例,您需要执行如下命令: ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` 通常您并不需要所有的连接器插件,可以通过配置`config/plugin_config`来指定您所需要的插件,例如,您只需要`connector-console`插件,那么您可以修改plugin.properties配置文件如下: diff --git a/pom.xml b/pom.xml index 41854d78fce..7ca8e0652f6 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ - 2.3.6-SNAPSHOT + 2.3.7-SNAPSHOT 2.1.1 UTF-8 1.8 diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 8532f7cba43..161134511c8 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -24,9 +24,9 @@ protostuff-collectionschema-1.8.0.jar protostuff-core-1.8.0.jar protostuff-runtime-1.8.0.jar scala-library-2.12.15.jar -seatunnel-jackson-2.3.6-SNAPSHOT-optional.jar -seatunnel-guava-2.3.6-SNAPSHOT-optional.jar -seatunnel-hazelcast-shade-2.3.6-SNAPSHOT-optional.jar +seatunnel-jackson-2.3.7-SNAPSHOT-optional.jar +seatunnel-guava-2.3.7-SNAPSHOT-optional.jar +seatunnel-hazelcast-shade-2.3.7-SNAPSHOT-optional.jar slf4j-api-1.7.25.jar jsqlparser-4.5.jar animal-sniffer-annotations-1.17.jar @@ -44,4 +44,4 @@ accessors-smart-2.4.7.jar asm-9.1.jar avro-1.11.1.jar groovy-4.0.16.jar -seatunnel-janino-2.3.6-SNAPSHOT-optional.jar \ No newline at end of file +seatunnel-janino-2.3.7-SNAPSHOT-optional.jar \ No newline at end of file From 9d56cc33b4f62316ed694c63a088fd50c1f51628 Mon Sep 17 00:00:00 2001 From: virvle <55478661+virvle@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:51:26 +0800 Subject: [PATCH 17/32] Update Oracle-CDC.md (#7285) --- .github/workflows/update_build_status.yml | 2 +- docs/en/connector-v2/source/Oracle-CDC.md | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/update_build_status.yml b/.github/workflows/update_build_status.yml index 05cf4914a25..03718ba7472 100644 --- a/.github/workflows/update_build_status.yml +++ b/.github/workflows/update_build_status.yml @@ -105,4 +105,4 @@ jobs: } } } - } + } \ No newline at end of file diff --git a/docs/en/connector-v2/source/Oracle-CDC.md b/docs/en/connector-v2/source/Oracle-CDC.md index cedbda141f6..5d22aa1c4ae 100644 --- a/docs/en/connector-v2/source/Oracle-CDC.md +++ b/docs/en/connector-v2/source/Oracle-CDC.md @@ -91,6 +91,8 @@ GRANT SELECT ON V_$ARCHIVED_LOG TO logminer_user; GRANT SELECT ON V_$ARCHIVE_DEST_STATUS TO logminer_user; GRANT EXECUTE ON DBMS_LOGMNR TO logminer_user; GRANT EXECUTE ON DBMS_LOGMNR_D TO logminer_user; +GRANT SELECT ANY TRANSACTION TO logminer_user; +GRANT SELECT ON V_$TRANSACTION TO logminer_user; ``` ##### Oracle 11g is not supported From 73632bad2b93e6879c673e0f00bb83035aa51408 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 6 Aug 2024 10:08:45 +0800 Subject: [PATCH 18/32] [Hotfix][Zeta] Fix task cannot be stopped when system is busy (#7292) --- .../engine/server/TaskExecutionService.java | 22 ++++++++++++++----- .../engine/server/master/JobMaster.java | 17 +++++++++++++- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java index 94f0fa324fc..00716f2c90a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java @@ -55,6 +55,7 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.hazelcast.core.OperationTimeoutException; import com.hazelcast.instance.impl.NodeState; import com.hazelcast.internal.metrics.DynamicMetricsProvider; import com.hazelcast.internal.metrics.MetricDescriptor; @@ -624,9 +625,12 @@ private void updateMetricsContextInImap() { }); }); if (localMap.size() > 0) { + boolean lockedIMap = false; try { - if (!metricsImap.tryLock( - Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS)) { + lockedIMap = + metricsImap.tryLock( + Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS); + if (!lockedIMap) { logger.warning("try lock failed in update metrics"); return; } @@ -640,10 +644,16 @@ private void updateMetricsContextInImap() { "The Imap acquisition failed due to the hazelcast node being offline or restarted, and will be retried next time", e); } finally { - try { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); - } catch (Throwable e) { - logger.warning("unlock imap failed in update metrics", e); + if (lockedIMap) { + boolean unLockedIMap = false; + while (!unLockedIMap) { + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + unLockedIMap = true; + } catch (OperationTimeoutException e) { + logger.warning("unlock imap failed in update metrics", e); + } + } } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index aa74460b056..888114bec95 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -72,6 +72,7 @@ import com.hazelcast.cluster.Address; import com.hazelcast.core.HazelcastInstanceNotActiveException; +import com.hazelcast.core.OperationTimeoutException; import com.hazelcast.flakeidgen.FlakeIdGenerator; import com.hazelcast.internal.serialization.Data; import com.hazelcast.jet.datamodel.Tuple2; @@ -674,8 +675,12 @@ public void removeMetricsContext( if ((pipelineStatus.equals(PipelineStatus.FINISHED) && !checkpointManager.isPipelineSavePointEnd(pipelineLocation)) || pipelineStatus.equals(PipelineStatus.CANCELED)) { + + boolean lockedIMap = false; try { metricsImap.lock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + lockedIMap = true; + HashMap centralMap = metricsImap.get(Constant.IMAP_RUNNING_JOB_METRICS_KEY); if (centralMap != null) { @@ -693,7 +698,17 @@ public void removeMetricsContext( metricsImap.put(Constant.IMAP_RUNNING_JOB_METRICS_KEY, centralMap); } } finally { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + if (lockedIMap) { + boolean unLockedIMap = false; + while (!unLockedIMap) { + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + unLockedIMap = true; + } catch (OperationTimeoutException e) { + LOGGER.warning("unlock imap failed in update metrics", e); + } + } + } } } } From c94ea325b7a70ebbfc3eabdaa5025c6d0292384c Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 6 Aug 2024 11:23:52 +0800 Subject: [PATCH 19/32] [Fix][Doc] Fix miss sink-options-placeholders.md in sidebars (#7310) --- docs/sidebars.js | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sidebars.js b/docs/sidebars.js index 1a9a1cf6ec5..33655a48171 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -90,6 +90,7 @@ const sidebars = { "concept/connector-v2-features", 'concept/schema-feature', 'concept/JobEnvConfig', + 'concept/sink-options-placeholders', 'concept/sql-config', 'concept/speed-limit', 'concept/event-listener' From bb2c912404fac13e829044c60259f2ab27bff3a1 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Tue, 6 Aug 2024 21:46:12 +0800 Subject: [PATCH 20/32] [Fix][Doris] Fix the abnormality of deleting data in CDC scenario. (#7315) --- .../serialize/SeaTunnelRowSerializer.java | 77 ++++---- .../doris/sink/writer/DorisStreamLoad.java | 32 +--- .../connector-doris-e2e/pom.xml | 14 ++ .../e2e/connector/doris/DorisCDCSinkIT.java | 171 ++++++++++++++++-- .../src/test/resources/ddl/mysql_cdc.sql | 38 ++++ .../test/resources/docker/server-gtids/my.cnf | 65 +++++++ .../src/test/resources/docker/setup.sql | 28 +++ .../write-cdc-changelog-to-doris.conf | 18 +- 8 files changed, 356 insertions(+), 87 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java index 0c5b9c0c420..0e67257a32e 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonGenerator; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.api.serialization.SerializationSchema; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -29,6 +30,7 @@ import org.apache.seatunnel.format.text.TextSerializationSchema; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -42,6 +44,7 @@ public class SeaTunnelRowSerializer implements DorisSerializer { private final SeaTunnelRowType seaTunnelRowType; private final String fieldDelimiter; private final boolean enableDelete; + private final SerializationSchema serialize; public SeaTunnelRowSerializer( String type, @@ -49,32 +52,46 @@ public SeaTunnelRowSerializer( String fieldDelimiter, boolean enableDelete) { this.type = type; - this.seaTunnelRowType = seaTunnelRowType; this.fieldDelimiter = fieldDelimiter; this.enableDelete = enableDelete; - } + List fieldNames = new ArrayList<>(Arrays.asList(seaTunnelRowType.getFieldNames())); + List> fieldTypes = + new ArrayList<>(Arrays.asList(seaTunnelRowType.getFieldTypes())); + + if (enableDelete) { + fieldNames.add(LoadConstants.DORIS_DELETE_SIGN); + fieldTypes.add(STRING_TYPE); + } - public byte[] buildJsonString(SeaTunnelRow row, SeaTunnelRowType seaTunnelRowType) - throws IOException { + this.seaTunnelRowType = + new SeaTunnelRowType( + fieldNames.toArray(new String[0]), + fieldTypes.toArray(new SeaTunnelDataType[0])); - JsonSerializationSchema jsonSerializationSchema = - new JsonSerializationSchema(seaTunnelRowType, NULL_VALUE); - ObjectMapper mapper = jsonSerializationSchema.getMapper(); - mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); - return jsonSerializationSchema.serialize(row); + if (JSON.equals(type)) { + JsonSerializationSchema jsonSerializationSchema = + new JsonSerializationSchema(this.seaTunnelRowType, NULL_VALUE); + ObjectMapper mapper = jsonSerializationSchema.getMapper(); + mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); + this.serialize = jsonSerializationSchema; + } else { + this.serialize = + TextSerializationSchema.builder() + .seaTunnelRowType(this.seaTunnelRowType) + .delimiter(fieldDelimiter) + .nullValue(NULL_VALUE) + .build(); + } } - public byte[] buildCSVString(SeaTunnelRow row, SeaTunnelRowType seaTunnelRowType) - throws IOException { + public byte[] buildJsonString(SeaTunnelRow row) { + + return serialize.serialize(row); + } - TextSerializationSchema build = - TextSerializationSchema.builder() - .seaTunnelRowType(seaTunnelRowType) - .delimiter(fieldDelimiter) - .nullValue(NULL_VALUE) - .build(); + public byte[] buildCSVString(SeaTunnelRow row) { - return build.serialize(row); + return serialize.serialize(row); } public String parseDeleteSign(RowKind rowKind) { @@ -93,29 +110,17 @@ public void open() throws IOException {} @Override public byte[] serialize(SeaTunnelRow seaTunnelRow) throws IOException { - List fieldNames = Arrays.asList(seaTunnelRowType.getFieldNames()); - List> fieldTypes = Arrays.asList(seaTunnelRowType.getFieldTypes()); - if (enableDelete) { - SeaTunnelRow seaTunnelRowEnableDelete = seaTunnelRow.copy(); - seaTunnelRowEnableDelete.setField( - seaTunnelRow.getFields().length, parseDeleteSign(seaTunnelRow.getRowKind())); - fieldNames.add(LoadConstants.DORIS_DELETE_SIGN); - fieldTypes.add(STRING_TYPE); + + List newFields = new ArrayList<>(Arrays.asList(seaTunnelRow.getFields())); + newFields.add(parseDeleteSign(seaTunnelRow.getRowKind())); + seaTunnelRow = new SeaTunnelRow(newFields.toArray()); } if (JSON.equals(type)) { - return buildJsonString( - seaTunnelRow, - new SeaTunnelRowType( - fieldNames.toArray(new String[0]), - fieldTypes.toArray(new SeaTunnelDataType[0]))); + return buildJsonString(seaTunnelRow); } else if (CSV.equals(type)) { - return buildCSVString( - seaTunnelRow, - new SeaTunnelRowType( - fieldNames.toArray(new String[0]), - fieldTypes.toArray(new SeaTunnelDataType[0]))); + return buildCSVString(seaTunnelRow); } else { throw new IllegalArgumentException("The type " + type + " is not supported!"); } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java index eadcf94cd56..40b75aedc61 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java @@ -17,7 +17,10 @@ package org.apache.seatunnel.connectors.doris.sink.writer; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.connectors.doris.config.DorisConfig; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; @@ -31,9 +34,9 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.util.EntityUtils; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import java.io.IOException; @@ -64,23 +67,23 @@ public class DorisStreamLoad implements Serializable { private static final String ABORT_URL_PATTERN = "http://%s/api/%s/_stream_load_2pc"; private static final String JOB_EXIST_FINISHED = "FINISHED"; private final String loadUrlStr; - private final String hostPort; + @Getter private final String hostPort; private final String abortUrlStr; private final String user; private final String passwd; - private final String db; + @Getter private final String db; private final String table; private final boolean enable2PC; private final boolean enableDelete; private final Properties streamLoadProp; private final RecordStream recordStream; - private Future pendingLoadFuture; + @Getter private Future pendingLoadFuture; private final CloseableHttpClient httpClient; private final ExecutorService executorService; private volatile boolean loadBatchFirstRecord; private volatile boolean loading = false; private String label; - private long recordCount = 0; + @Getter private long recordCount = 0; public DorisStreamLoad( String hostPort, @@ -115,18 +118,6 @@ public DorisStreamLoad( loadBatchFirstRecord = true; } - public String getDb() { - return db; - } - - public String getHostPort() { - return hostPort; - } - - public Future getPendingLoadFuture() { - return pendingLoadFuture; - } - public void abortPreCommit(String labelSuffix, long chkID) throws Exception { long startChkID = chkID; log.info("abort for labelSuffix {}. start chkId {}.", labelSuffix, chkID); @@ -196,10 +187,6 @@ public void writeRecord(byte[] record) throws IOException { recordCount++; } - public long getRecordCount() { - return recordCount; - } - public String getLoadFailedMsg() { if (!loading) { return null; @@ -300,10 +287,9 @@ public void abortTransaction(long txnID) throws Exception { "Fail to abort transaction " + txnID + " with url " + abortUrlStr); } - ObjectMapper mapper = new ObjectMapper(); String loadResult = EntityUtils.toString(response.getEntity()); Map res = - mapper.readValue(loadResult, new TypeReference>() {}); + JsonUtils.parseObject(loadResult, new TypeReference>() {}); if (!LoadStatus.SUCCESS.equals(res.get("status"))) { if (ResponseUtil.isCommitted(res.get("msg"))) { throw new DorisConnectorException( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml index af85d92acef..7a3008adb3a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml @@ -49,5 +49,19 @@ ${mysql.version} test + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + + org.testcontainers + mysql + ${testcontainer.version} + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java index 9afa91d4e81..33108b8b8eb 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java @@ -17,16 +17,27 @@ package org.apache.seatunnel.e2e.connector.doris; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; import lombok.extern.slf4j.Slf4j; +import java.sql.Connection; +import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -34,11 +45,18 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.awaitility.Awaitility.await; + @Slf4j -@Disabled("we need resolve the issue of network between containers") +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") public class DorisCDCSinkIT extends AbstractDorisIT { private static final String DATABASE = "test"; @@ -60,34 +78,121 @@ public class DorisCDCSinkIT extends AbstractDorisIT { + "\"replication_allocation\" = \"tag.location.default: 1\"" + ")"; + // mysql + private static final String MYSQL_HOST = "mysql_cdc_e2e"; + private static final String MYSQL_USER_NAME = "mysqluser"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; + private static final String MYSQL_DATABASE = "mysql_cdc"; + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); + private static final String SOURCE_TABLE = "mysql_cdc_e2e_source_table"; + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Doris-CDC/lib && cd /tmp/seatunnel/plugins/Doris-CDC/lib && wget " + + driverUrl()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase( + MYSQL_CONTAINER, MYSQL_DATABASE, "mysqluser", "mysqlpw", MYSQL_DATABASE); + + private static MySqlContainer createMySqlContainer(MySqlVersion version) { + return new MySqlContainer(version) + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("docker/setup.sql") + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER_NAME) + .withPassword(MYSQL_USER_PASSWORD) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("mysql-docker-image"))); + } + + private String driverUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + @BeforeAll public void init() { + log.info("The second stage: Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + inventoryDatabase.createAndInitialize(); + log.info("Mysql ddl execution is complete"); initializeJdbcTable(); } @TestTemplate public void testDorisCDCSink(TestContainer container) throws Exception { - Container.ExecResult execResult = - container.executeJob("/write-cdc-changelog-to-doris.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); + + clearTable(DATABASE, SINK_TABLE); + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/write-cdc-changelog-to-doris.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); String sinkSql = String.format("select * from %s.%s", DATABASE, SINK_TABLE); - Set> actual = new HashSet<>(); - try (Statement sinkStatement = jdbcConnection.createStatement()) { - ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); - while (sinkResultSet.next()) { - List row = - Arrays.asList( - sinkResultSet.getLong("uuid"), - sinkResultSet.getString("name"), - sinkResultSet.getInt("score")); - actual.add(row); - } - } + Set> expected = - Stream.>of(Arrays.asList(1L, "A_1", 100), Arrays.asList(3L, "C", 100)) + Stream.>of( + Arrays.asList(1L, "Alice", 95), Arrays.asList(2L, "Bob", 88)) .collect(Collectors.toSet()); - Assertions.assertIterableEquals(expected, actual); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Set> actual = new HashSet<>(); + try (Statement sinkStatement = jdbcConnection.createStatement()) { + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + while (sinkResultSet.next()) { + List row = + Arrays.asList( + sinkResultSet.getLong("uuid"), + sinkResultSet.getString("name"), + sinkResultSet.getInt("score")); + actual.add(row); + } + } + Assertions.assertIterableEquals(expected, actual); + }); + + executeSql("DELETE FROM " + MYSQL_DATABASE + "." + SOURCE_TABLE + " WHERE uuid = 1"); + + Set> expectedAfterDelete = + Stream.>of(Arrays.asList(2L, "Bob", 88)).collect(Collectors.toSet()); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Set> actual = new HashSet<>(); + try (Statement sinkStatement = jdbcConnection.createStatement()) { + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + while (sinkResultSet.next()) { + List row = + Arrays.asList( + sinkResultSet.getLong("uuid"), + sinkResultSet.getString("name"), + sinkResultSet.getInt("score")); + actual.add(row); + } + } + Assertions.assertIterableEquals(expectedAfterDelete, actual); + }); + executeSql( + "INSERT INTO " + MYSQL_DATABASE + "." + SOURCE_TABLE + " VALUES (1, 'Alice', 95)"); } private void initializeJdbcTable() { @@ -100,4 +205,32 @@ private void initializeJdbcTable() { throw new RuntimeException("Initializing table failed!", e); } } + + private void executeDorisSql(String sql) { + try (Statement statement = jdbcConnection.createStatement()) { + statement.execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + } + + // Execute SQL + private void executeSql(String sql) { + try (Connection connection = getJdbcConnection()) { + connection.createStatement().execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void clearTable(String database, String tableName) { + executeDorisSql("truncate table " + database + "." + tableName); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql new file mode 100644 index 00000000000..638da2981b3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql @@ -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. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: inventory +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE IF NOT EXISTS `mysql_cdc`; + +use mysql_cdc; +-- Create a mysql data source table +CREATE TABLE IF NOT EXISTS `mysql_cdc`.`mysql_cdc_e2e_source_table` ( + `uuid` BIGINT, + `name` VARCHAR(128), + `score` INT, + PRIMARY KEY (`uuid`) +) ENGINE=InnoDB; + + + +truncate table `mysql_cdc`.`mysql_cdc_e2e_source_table`; + +INSERT INTO `mysql_cdc`.`mysql_cdc_e2e_source_table` (uuid, name, score) VALUES +(1, 'Alice', 95), +(2, 'Bob', 88); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 00000000000..a390897885d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf @@ -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. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql new file mode 100644 index 00000000000..429061558ba --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql @@ -0,0 +1,28 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'mysqluser' - all privileges +-- 2) 'st_user_source' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; + +CREATE USER 'st_user_source' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user_source'@'%'; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf index d4d4e69f9d6..7e811c709b3 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf @@ -17,23 +17,24 @@ env { parallelism = 1 - job.mode = "BATCH" + job.mode = "STREAMING" + checkpoint.interval = 5000 } source { MySQL-CDC { parallelism = 1 - server-id = 5656 - username = "root" - password = "Bigdata2023@" - table-names = ["test.e2e_table_sink"] - base-url = "jdbc:mysql://119.3.230.145:56725/test" + server-id = 5652 + username = "st_user_source" + password = "mysqlpw" + table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] + base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" } } sink { Doris { - fenodes = "10.16.10.14:8234" + fenodes = "doris_e2e:8030" username = root password = "" database = "test" @@ -43,8 +44,7 @@ sink { sink.enable-delete = "true" doris.config { format = "csv" - "column_separator" = "\\x01" - "line_delimiter" = "\\x01" + "column_separator" = "," } } } \ No newline at end of file From f0cefbeb4ac856f1eca17f068b2549d750f321ee Mon Sep 17 00:00:00 2001 From: Arin <136636751+asapekia@users.noreply.github.com> Date: Tue, 6 Aug 2024 19:23:38 +0530 Subject: [PATCH 21/32] [Feature] [Activemq] Added activemq sink (#7251) --- .../workflows/labeler/label-scope-conf.yml | 5 + config/plugin_config | 1 + docs/en/connector-v2/sink/Activemq.md | 123 +++++++++ plugin-mapping.properties | 1 + .../connector-activemq/pom.xml | 65 +++++ .../activemq/client/ActivemqClient.java | 156 +++++++++++ .../activemq/config/ActivemqConfig.java | 242 ++++++++++++++++++ .../exception/ActivemqConnectorErrorCode.java | 51 ++++ .../exception/ActivemqConnectorException.java | 36 +++ .../seatunnel/activemq/sink/ActivemqSink.java | 48 ++++ .../activemq/sink/ActivemqSinkFactory.java | 83 ++++++ .../activemq/sink/ActivemqSinkWriter.java | 56 ++++ .../activemq/ActivemqFactoryTest.java | 31 +++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 7 + .../connector-activemq-e2e/pom.xml | 60 +++++ .../e2e/connector/activemq/ActivemqIT.java | 118 +++++++++ .../src/test/resources/e2e.json | 100 ++++++++ .../test/resources/fake_source_to_sink.conf | 117 +++++++++ .../resources/localfile_source_to_sink.conf | 117 +++++++++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 21 files changed, 1419 insertions(+) create mode 100644 docs/en/connector-v2/sink/Activemq.md create mode 100644 seatunnel-connectors-v2/connector-activemq/pom.xml create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index 7db40f5ec5d..599ed649396 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -252,6 +252,11 @@ Milvus: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-milvus/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(milvus)/**' +activemq: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-activemq/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(activemq)/**' 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 d80d2e6ab06..f6549168d6d 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -86,4 +86,5 @@ connector-rocketmq connector-tdengine connector-web3j connector-milvus +connector-activemq --end-- \ No newline at end of file diff --git a/docs/en/connector-v2/sink/Activemq.md b/docs/en/connector-v2/sink/Activemq.md new file mode 100644 index 00000000000..3151585d082 --- /dev/null +++ b/docs/en/connector-v2/sink/Activemq.md @@ -0,0 +1,123 @@ +# Activemq + +> Activemq sink connector + +## Description + +Used to write data to Activemq. + +## Key features + +- [ ] [exactly-once](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|-------------------------------------|---------|----------|---------------| +| host | string | no | - | +| port | int | no | - | +| virtual_host | string | no | - | +| username | string | no | - | +| password | string | no | - | +| queue_name | string | yes | - | +| uri | string | yes | - | +| check_for_duplicate | boolean | no | - | +| client_id | boolean | no | - | +| copy_message_on_send | boolean | no | - | +| disable_timeStamps_by_default | boolean | no | - | +| use_compression | boolean | no | - | +| always_session_async | boolean | no | - | +| dispatch_async | boolean | no | - | +| nested_map_and_list_enabled | boolean | no | - | +| warnAboutUnstartedConnectionTimeout | boolean | no | - | +| closeTimeout | int | no | - | + +### host [string] + +the default host to use for connections + +### port [int] + +the default port to use for connections + +### username [string] + +the AMQP user name to use when connecting to the broker + +### password [string] + +the password to use when connecting to the broker + +### uri [string] + +convenience method for setting the fields in an AMQP URI: host, port, username, password and virtual host + +### queue_name [string] + +the queue to write the message to + +### check_for_duplicate [boolean] + +will check for duplucate messages + +### client_id [string] + +client id + +### copy_message_on_send [boolean] + +if true, enables new JMS Message object as part of the send method + +### disable_timeStamps_by_default [boolean] + +disables timestamp for slight performance boost + +### use_compression [boolean] + +Enables the use of compression on the message’s body. + +### always_session_async [boolean] + +When true a separate thread is used for dispatching messages for each Session in the Connection. + +### always_sync_send [boolean] + +When true a MessageProducer will always use Sync sends when sending a Message + +### close_timeout [boolean] + +Sets the timeout, in milliseconds, before a close is considered complete. + +### dispatch_async [boolean] + +Should the broker dispatch messages asynchronously to the consumer + +### nested_map_and_list_enabled [boolean] + +Controls whether Structured Message Properties and MapMessages are supported + +### warn_about_unstarted_connection_timeout [int] + +The timeout, in milliseconds, from the time of connection creation to when a warning is generated + +## Example + +simple: + +```hocon +sink { + ActiveMQ { + uri="tcp://localhost:61616" + username = "admin" + password = "admin" + queue_name = "test1" + } +} +``` + +## Changelog + +### next version + +- Add Activemq Source Connector + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 9936afcbaaf..1942f875d7c 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -129,3 +129,4 @@ seatunnel.source.ObsFile = connector-file-obs seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus +seatunnel.sink.ActiveMQ = connector-activemq \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-activemq/pom.xml b/seatunnel-connectors-v2/connector-activemq/pom.xml new file mode 100644 index 00000000000..7a72a3b1c4c --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/pom.xml @@ -0,0 +1,65 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-activemq + SeaTunnel : Connectors V2 : Activemq + + + 5.14.5 + + + + org.apache.seatunnel + connector-common + ${project.version} + + + org.apache.activemq + activemq-client + ${activemq.version} + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + org.apache.seatunnel + seatunnel-format-text + ${project.version} + + + + diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java new file mode 100644 index 00000000000..f4983d35dbd --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java @@ -0,0 +1,156 @@ +/* + * 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.activemq.client; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.activemq.exception.ActivemqConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.activemq.exception.ActivemqConnectorException; + +import org.apache.activemq.ActiveMQConnectionFactory; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import javax.jms.Connection; +import javax.jms.Destination; +import javax.jms.JMSException; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; + +import java.nio.charset.StandardCharsets; + +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SESSION_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SYNC_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CHECK_FOR_DUPLICATE; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLIENT_ID; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLOSE_TIMEOUT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CONSUMER_EXPIRY_CHECK_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISPATCH_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.NESTED_MAP_AND_LIST_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PASSWORD; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.QUEUE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.URI; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USERNAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT; + +@Slf4j +@AllArgsConstructor +public class ActivemqClient { + private final ReadonlyConfig config; + private final ActiveMQConnectionFactory connectionFactory; + private final Connection connection; + + public ActivemqClient(ReadonlyConfig config) { + this.config = config; + try { + this.connectionFactory = getConnectionFactory(); + log.info("connection factory created"); + this.connection = createConnection(config); + log.info("connection created"); + + } catch (Exception e) { + e.printStackTrace(); + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.CREATE_ACTIVEMQ_CLIENT_FAILED, + "Error while create AMQ client "); + } + } + + public ActiveMQConnectionFactory getConnectionFactory() { + log.info("broker url : " + config.get(URI)); + ActiveMQConnectionFactory factory = new ActiveMQConnectionFactory(config.get(URI)); + + if (config.get(ALWAYS_SESSION_ASYNC) != null) { + factory.setAlwaysSessionAsync(config.get(ALWAYS_SESSION_ASYNC)); + } + + if (config.get(CLIENT_ID) != null) { + factory.setClientID(config.get(CLIENT_ID)); + } + + if (config.get(ALWAYS_SYNC_SEND) != null) { + factory.setAlwaysSyncSend(config.get(ALWAYS_SYNC_SEND)); + } + + if (config.get(CHECK_FOR_DUPLICATE) != null) { + factory.setCheckForDuplicates(config.get(CHECK_FOR_DUPLICATE)); + } + + if (config.get(CLOSE_TIMEOUT) != null) { + factory.setCloseTimeout(config.get(CLOSE_TIMEOUT)); + } + + if (config.get(CONSUMER_EXPIRY_CHECK_ENABLED) != null) { + factory.setConsumerExpiryCheckEnabled(config.get(CONSUMER_EXPIRY_CHECK_ENABLED)); + } + if (config.get(DISPATCH_ASYNC) != null) { + factory.setDispatchAsync(config.get(DISPATCH_ASYNC)); + } + + if (config.get(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT) != null) { + factory.setWarnAboutUnstartedConnectionTimeout( + config.get(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT)); + } + + if (config.get(NESTED_MAP_AND_LIST_ENABLED) != null) { + factory.setNestedMapAndListEnabled(config.get(NESTED_MAP_AND_LIST_ENABLED)); + } + return factory; + } + + public void write(byte[] msg) { + try { + this.connection.start(); + Session session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + Destination destination = session.createQueue(config.get(QUEUE_NAME)); + MessageProducer producer = session.createProducer(destination); + String messageBody = new String(msg, StandardCharsets.UTF_8); + TextMessage objectMessage = session.createTextMessage(messageBody); + producer.send(objectMessage); + + } catch (JMSException e) { + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.SEND_MESSAGE_FAILED, + String.format( + "Cannot send AMQ message %s at %s", + config.get(QUEUE_NAME), config.get(CLIENT_ID)), + e); + } + } + + public void close() { + try { + if (connection != null) { + connection.close(); + } + } catch (JMSException e) { + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.CLOSE_CONNECTION_FAILED, + String.format( + "Error while closing AMQ connection with %s", config.get(QUEUE_NAME))); + } + } + + private Connection createConnection(ReadonlyConfig config) throws JMSException { + if (config.get(USERNAME) != null && config.get(PASSWORD) != null) { + return connectionFactory.createConnection(config.get(USERNAME), config.get(PASSWORD)); + } + return connectionFactory.createConnection(); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java new file mode 100644 index 00000000000..868ac40a0c8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java @@ -0,0 +1,242 @@ +/* + * 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.activemq.config; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import com.google.common.annotations.VisibleForTesting; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +@Setter +@Getter +@AllArgsConstructor +public class ActivemqConfig implements Serializable { + private String host; + private Integer port; + private String username; + private String password; + private String uri; + private String queueName; + private Boolean checkForDuplicate; + private String clientID; + private Integer closeTimeout; + private Boolean consumerExpiryCheckEnabled; + private Boolean copyMessageOnSend; + private Boolean disableTimeStampsByDefault; + private Boolean dispatchAsync; + private Boolean nestedMapAndListEnabled; + private Boolean useCompression; + private Boolean alwaysSessionAsync; + private Boolean alwaysSyncSend; + private Integer warnAboutUnstartedConnectionTimeout; + + private final Map sinkOptionProps = new HashMap<>(); + + public static final Option HOST = + Options.key("host") + .stringType() + .noDefaultValue() + .withDescription("the default host to use for connections"); + + public static final Option PORT = + Options.key("port") + .intType() + .noDefaultValue() + .withDescription("the default port to use for connections"); + + public static final Option USERNAME = + Options.key("username") + .stringType() + .noDefaultValue() + .withDescription("the AMQP user name to use when connecting to the broker"); + + public static final Option PASSWORD = + Options.key("password") + .stringType() + .noDefaultValue() + .withDescription("the password to use when connecting to the broker"); + + public static final Option QUEUE_NAME = + Options.key("queue_name") + .stringType() + .noDefaultValue() + .withDescription("the queue to write the message to"); + + public static final Option URI = + Options.key("uri") + .stringType() + .noDefaultValue() + .withDescription( + "convenience method for setting the fields in an AMQP URI: host, port, username, password and virtual host"); + + public static final Option CHECK_FOR_DUPLICATE = + Options.key("check_for_duplicate") + .booleanType() + .noDefaultValue() + .withDescription( + "When true the consumer will check for duplicate messages and properly handle +" + + "the message to make sure that it is not processed twice inadvertently."); + public static final Option CLIENT_ID = + Options.key("client_id") + .stringType() + .noDefaultValue() + .withDescription("Sets the JMS clientID to use for the connection."); + + public static final Option COPY_MESSAGE_ON_SEND = + Options.key("copy_message_on_send") + .booleanType() + .noDefaultValue() + .withDescription( + "Should a JMS message be copied to a new JMS Message object as part of the send() method in JMS. " + + "This is enabled by default to be compliant with the JMS specification. " + + "For a performance boost set to false if you do not mutate JMS messages after they are sent."); + + public static final Option DISABLE_TIMESTAMP_BY_DEFAULT = + Options.key("disable_timeStamps_by_default") + .booleanType() + .noDefaultValue() + .withDescription( + "Sets whether or not timestamps on messages should be disabled or not. " + + "For a small performance boost set to false."); + + public static final Option USE_COMPRESSION = + Options.key("use_compression") + .booleanType() + .noDefaultValue() + .withDescription("Enables the use of compression on the message’s body."); + + public static final Option ALWAYS_SESSION_ASYNC = + Options.key("always_session_async") + .booleanType() + .noDefaultValue() + .withDescription( + "When true a separate thread is used for dispatching messages for each Session in the Connection. " + + "A separate thread is always used when there’s more than one session, " + + "or the session isn’t in Session.AUTO_ACKNOWLEDGE or Session.DUPS_OK_ACKNOWLEDGE mode."); + + public static final Option ALWAYS_SYNC_SEND = + Options.key("always_sync_send") + .booleanType() + .noDefaultValue() + .withDescription( + "When true a MessageProducer will always use Sync sends when sending a Message " + + "even if it is not required for the Delivery Mode."); + + public static final Option CLOSE_TIMEOUT = + Options.key("close_timeout") + .intType() + .noDefaultValue() + .withDescription( + "Sets the timeout, in milliseconds, before a close is considered complete. " + + "Normally a close() on a connection waits for confirmation from the broker. " + + "This allows the close operation to timeout preventing the client from hanging when no broker is available."); + + public static final Option DISPATCH_ASYNC = + Options.key("dispatch_async") + .booleanType() + .noDefaultValue() + .withDescription( + "Should the broker dispatch messages asynchronously to the consumer?"); + + public static final Option NESTED_MAP_AND_LIST_ENABLED = + Options.key("nested_map_and_list_enabled") + .booleanType() + .noDefaultValue() + .withDescription( + "Controls whether Structured Message Properties and MapMessages are supported " + + "so that Message properties and MapMessage entries can contain nested Map and List objects." + + " Available from version 4.1."); + + public static final Option WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT = + Options.key("warn_about_unstarted_connection_timeout") + .intType() + .noDefaultValue() + .withDescription( + "The timeout, in milliseconds, from the time of connection creation to when a warning is generated " + + "if the connection is not properly started via Connection.start() and a message is received by a consumer. " + + "It is a very common gotcha to forget to start the connection and then wonder why no messages are delivered " + + "so this option makes the default case to create a warning if the user forgets. " + + "To disable the warning just set the value to < 0."); + + public static final Option CONSUMER_EXPIRY_CHECK_ENABLED = + Options.key("consumer_expiry_check_enabled") + .booleanType() + .noDefaultValue() + .withDescription( + "Controls whether message expiration checking is done in each " + + "MessageConsumer prior to dispatching a message."); + + public ActivemqConfig(Config config) { + this.host = config.getString(HOST.key()); + this.port = config.getInt(PORT.key()); + this.queueName = config.getString(QUEUE_NAME.key()); + this.uri = config.getString(URI.key()); + if (config.hasPath(USERNAME.key())) { + this.username = config.getString(USERNAME.key()); + } + if (config.hasPath(PASSWORD.key())) { + this.password = config.getString(PASSWORD.key()); + } + if (config.hasPath(CHECK_FOR_DUPLICATE.key())) { + this.checkForDuplicate = config.getBoolean(CHECK_FOR_DUPLICATE.key()); + } + if (config.hasPath(CLIENT_ID.key())) { + this.clientID = config.getString(CLIENT_ID.key()); + } + if (config.hasPath(COPY_MESSAGE_ON_SEND.key())) { + this.copyMessageOnSend = config.getBoolean(COPY_MESSAGE_ON_SEND.key()); + } + if (config.hasPath(DISABLE_TIMESTAMP_BY_DEFAULT.key())) { + this.disableTimeStampsByDefault = config.getBoolean(DISABLE_TIMESTAMP_BY_DEFAULT.key()); + } + if (config.hasPath(USE_COMPRESSION.key())) { + this.useCompression = config.getBoolean(USE_COMPRESSION.key()); + } + if (config.hasPath(ALWAYS_SESSION_ASYNC.key())) { + this.alwaysSessionAsync = config.getBoolean(ALWAYS_SESSION_ASYNC.key()); + } + if (config.hasPath(ALWAYS_SYNC_SEND.key())) { + this.alwaysSyncSend = config.getBoolean(ALWAYS_SYNC_SEND.key()); + } + if (config.hasPath(CLOSE_TIMEOUT.key())) { + this.closeTimeout = config.getInt(CLOSE_TIMEOUT.key()); + } + if (config.hasPath(DISPATCH_ASYNC.key())) { + this.dispatchAsync = config.getBoolean(DISPATCH_ASYNC.key()); + } + if (config.hasPath(NESTED_MAP_AND_LIST_ENABLED.key())) { + this.nestedMapAndListEnabled = config.getBoolean(NESTED_MAP_AND_LIST_ENABLED.key()); + } + if (config.hasPath(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT.key())) { + this.warnAboutUnstartedConnectionTimeout = + config.getInt(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT.key()); + } + } + + @VisibleForTesting + public ActivemqConfig() {} +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java new file mode 100644 index 00000000000..138a49cdcb1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum ActivemqConnectorErrorCode implements SeaTunnelErrorCode { + HANDLE_SHUTDOWN_SIGNAL_FAILED("ACTIVEMQ-01", "handle queue consumer shutdown signal failed"), + CREATE_ACTIVEMQ_CLIENT_FAILED("ACTIVEMQ-02", "create activemq client failed"), + CLOSE_CONNECTION_FAILED("ACTIVEMQ-03", "close connection failed"), + SEND_MESSAGE_FAILED("ACTIVEMQ-04", "send messages failed"), + MESSAGE_ACK_FAILED( + "ACTIVEMQ-05", "messages could not be acknowledged during checkpoint creation"), + MESSAGE_ACK_REJECTED("ACTIVEMQ-06", "messages could not be acknowledged with basicReject"), + PARSE_URI_FAILED("ACTIVEMQ-07", "parse uri failed"), + INIT_SSL_CONTEXT_FAILED("ACTIVEMQ-08", "initialize ssl context failed"), + SETUP_SSL_FACTORY_FAILED("ACTIVEMQ-09", "setup ssl factory failed"); + + private final String code; + private final String description; + + ActivemqConnectorErrorCode(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-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java new file mode 100644 index 00000000000..7791c8e6184 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java @@ -0,0 +1,36 @@ +/* + * 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.activemq.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class ActivemqConnectorException extends SeaTunnelRuntimeException { + public ActivemqConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public ActivemqConnectorException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } + + public ActivemqConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { + super(seaTunnelErrorCode, cause); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java new file mode 100644 index 00000000000..d1d37017959 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java @@ -0,0 +1,48 @@ +/* + * 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.activemq.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; + +import java.io.IOException; + +public class ActivemqSink extends AbstractSimpleSink { + private final SeaTunnelRowType seaTunnelRowType; + private final ReadonlyConfig pluginConfig; + + @Override + public String getPluginName() { + return "ActiveMQ"; + } + + public ActivemqSink(ReadonlyConfig pluginConfig, SeaTunnelRowType rowType) { + this.pluginConfig = pluginConfig; + this.seaTunnelRowType = rowType; + } + + @Override + public AbstractSinkWriter createWriter(SinkWriter.Context context) + throws IOException { + return new ActivemqSinkWriter(pluginConfig, seaTunnelRowType); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java new file mode 100644 index 00000000000..7f0dca38f6a --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SESSION_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SYNC_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CHECK_FOR_DUPLICATE; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLIENT_ID; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLOSE_TIMEOUT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.COPY_MESSAGE_ON_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISABLE_TIMESTAMP_BY_DEFAULT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISPATCH_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.HOST; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.NESTED_MAP_AND_LIST_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PASSWORD; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PORT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.QUEUE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.URI; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USERNAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USE_COMPRESSION; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT; + +@AutoService(Factory.class) +public class ActivemqSinkFactory implements TableSinkFactory { + + @Override + public String factoryIdentifier() { + return "ActiveMQ"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(QUEUE_NAME, URI) + .bundled(USERNAME, PASSWORD) + .optional( + HOST, + PORT, + CLIENT_ID, + CHECK_FOR_DUPLICATE, + COPY_MESSAGE_ON_SEND, + DISABLE_TIMESTAMP_BY_DEFAULT, + USE_COMPRESSION, + ALWAYS_SESSION_ASYNC, + ALWAYS_SYNC_SEND, + CLOSE_TIMEOUT, + DISPATCH_ASYNC, + NESTED_MAP_AND_LIST_ENABLED, + WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + return () -> + new ActivemqSink( + context.getOptions(), + context.getCatalogTable().getTableSchema().toPhysicalRowDataType()); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java new file mode 100644 index 00000000000..f3395552c44 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java @@ -0,0 +1,56 @@ +/* + * 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.activemq.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.activemq.client.ActivemqClient; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.format.json.JsonSerializationSchema; + +import java.util.Optional; + +public class ActivemqSinkWriter extends AbstractSinkWriter { + private ActivemqClient activeMQClient; + + private final SerializationSchema serializationSchema; + + public ActivemqSinkWriter(ReadonlyConfig config, SeaTunnelRowType seaTunnelRowType) { + this.activeMQClient = new ActivemqClient(config); + this.serializationSchema = new JsonSerializationSchema(seaTunnelRowType); + } + + @Override + public void write(SeaTunnelRow element) { + activeMQClient.write(serializationSchema.serialize(element)); + } + + @Override + public Optional prepareCommit() { + return Optional.empty(); + } + + @Override + public void close() { + if (activeMQClient != null) { + activeMQClient.close(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java b/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java new file mode 100644 index 00000000000..90732d8a0ed --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.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.activemq; + +import org.apache.seatunnel.connectors.seatunnel.activemq.sink.ActivemqSinkFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class ActivemqFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new ActivemqSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 68274736f08..e0564a5572b 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -78,6 +78,7 @@ connector-easysearch connector-web3j connector-milvus + connector-activemq diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index c96bf0b612b..a16d86cad5a 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -583,6 +583,13 @@ provided + + org.apache.seatunnel + connector-activemq + ${project.version} + provided + + com.aliyun.phoenix diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml new file mode 100644 index 00000000000..d94d24b29d4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + connector-activemq-e2e + SeaTunnel : E2E : Connector V2 : ActiveMQ + + + + + org.testcontainers + activemq + 1.20.1 + test + + + + org.apache.seatunnel + connector-common + ${project.version} + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-file-local + ${project.version} + + + + org.apache.seatunnel + connector-activemq + ${project.version} + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java new file mode 100644 index 00000000000..56b1faeab86 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java @@ -0,0 +1,118 @@ +/* + * 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.activemq; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.apache.activemq.ActiveMQConnectionFactory; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.HostPortWaitStrategy; +import org.testcontainers.utility.DockerImageName; + +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.JMSException; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Queue; +import javax.jms.Session; +import javax.jms.TextMessage; + +import java.io.IOException; +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class ActivemqIT extends TestSuiteBase { + + private static final String ACTIVEMQ_CONTAINER_HOST = "activemq-host"; + public GenericContainer activeMQContainer = + new GenericContainer<>(DockerImageName.parse("rmohr/activemq")) + .withExposedPorts(61616) + .withNetworkAliases(ACTIVEMQ_CONTAINER_HOST) + .withNetwork(NETWORK); + + private Connection connection; + private Session session; + private MessageProducer producer; + private MessageConsumer consumer; + + @BeforeAll + public void setup() throws JMSException, InterruptedException { + activeMQContainer + .withNetwork(NETWORK) + .waitingFor(new HostPortWaitStrategy().withStartupTimeout(Duration.ofMinutes(2))); + activeMQContainer.start(); + String brokerUrl = "tcp://127.0.0.1:" + activeMQContainer.getMappedPort(61616); + ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(brokerUrl); + connection = connectionFactory.createConnection(); + connection.start(); + + // Creating session for sending messages + session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + + // Getting the queue + Queue queue = session.createQueue("testQueue"); + + // Creating the producer & consumer + producer = session.createProducer(queue); + consumer = session.createConsumer(queue); + } + + @AfterAll + public void tearDown() throws JMSException { + // Cleaning up resources + if (producer != null) producer.close(); + if (session != null) session.close(); + if (connection != null) connection.close(); + } + + @Test + public void testSendMessage() throws JMSException { + String dummyPayload = "Dummy payload"; + + // Sending a text message to the queue + TextMessage message = session.createTextMessage(dummyPayload); + producer.send(message); + + // Receiving the message from the queue + TextMessage receivedMessage = (TextMessage) consumer.receive(5000); + + assertEquals(dummyPayload, receivedMessage.getText()); + } + + @TestTemplate + public void testSinkApacheActivemq(TestContainer container) + throws IOException, InterruptedException, JMSException { + Container.ExecResult execResult = container.executeJob("/fake_source_to_sink.conf"); + TextMessage textMessage = (TextMessage) consumer.receive(); + Assertions.assertTrue(textMessage.getText().contains("map")); + Assertions.assertTrue(textMessage.getText().contains("c_boolean")); + Assertions.assertTrue(textMessage.getText().contains("c_tinyint")); + Assertions.assertTrue(textMessage.getText().contains("c_timestamp")); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json new file mode 100644 index 00000000000..040ee633bd8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json @@ -0,0 +1,100 @@ +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}} +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf new file mode 100644 index 00000000000..a3c0859ed14 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf @@ -0,0 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +###### +###### This config file is a demonstration of batch processing in SeaTunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties + #job.mode = BATCH + job.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + + + + # You can also use other input plugins, such as hdfs + # hdfs { + # result_table_name = "accesslog" + # path = "hdfs://hadoop-cluster-01/nginx/accesslog" + # format = "json" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of input plugins, + # please go to https://seatunnel.apache.org/docs/category/source-v2 + + +transform { + # split data by specific delimiter + + # you can also use other transform plugins, such as sql + + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/category/transform-v2 +} + + + +sink { + ActiveMQ { + host = "activemq-e2e" + port = "5672" + queue_name = "testQueue" + uri="tcp://activemq-host:61616" + } +} + + # you can also you other output plugins, such as sql + # hdfs { + # path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed" + # save_mode = "append" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of output plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf new file mode 100644 index 00000000000..7c5757c51f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf @@ -0,0 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +###### +###### This config file is a demonstration of batch processing in SeaTunnel config +###### + +env { + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties + #job.mode = BATCH + job.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + LocalFile { + path = "/e2e.json" + file_format_type = "json" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + result_table_name = "fake" + } +} + + # You can also use other input plugins, such as hdfs + # hdfs { + # result_table_name = "accesslog" + # path = "hdfs://hadoop-cluster-01/nginx/accesslog" + # format = "json" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of input plugins, + # please go to https://seatunnel.apache.org/docs/category/source-v2 + + +transform { + # split data by specific delimiter + + # you can also use other transform plugins, such as sql + + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/category/transform-v2 +} + + + +sink { + ActiveMQ { + host = "active-e2e" + port = "5672" + username = "guest" + password = "guest" + queue_name = "test1" + uri="tcp://localhost:61616" + } +} + + # you can also you other output plugins, such as sql + # hdfs { + # path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed" + # save_mode = "append" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of output plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 0a0f909e199..2db67f88147 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -75,6 +75,7 @@ connector-hive-e2e connector-hudi-e2e connector-milvus-e2e + connector-activemq-e2e From 855254e737051edbaf4ca08b95ca010fe18fd214 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 7 Aug 2024 11:03:06 +0800 Subject: [PATCH 22/32] [Feature][Transform] Add LLM transform (#7303) * [Feature][Transform] Add LLM transform * update * update * retrigger --- docs/en/transform-v2/llm.md | 122 ++++++++++++++++++ docs/zh/transform-v2/llm.md | 120 +++++++++++++++++ .../seatunnel/e2e/transform/TestLLMIT.java | 90 +++++++++++++ .../test/resources/llm_openai_transform.conf | 75 +++++++++++ .../src/test/resources/mockserver-config.json | 40 ++++++ seatunnel-transforms-v2/pom.xml | 15 +++ .../common/SeaTunnelRowAccessor.java | 4 + .../seatunnel/transform/llm/LLMTransform.java | 119 +++++++++++++++++ .../transform/llm/LLMTransformConfig.java | 71 ++++++++++ .../transform/llm/LLMTransformFactory.java | 59 +++++++++ .../transform/llm/ModelProvider.java | 22 ++++ .../transform/llm/model/AbstractModel.java | 69 ++++++++++ .../seatunnel/transform/llm/model/Model.java | 29 +++++ .../llm/model/openai/OpenAIModel.java | 104 +++++++++++++++ .../transform/LLMTransformFactoryTest.java | 32 +++++ .../transform/llm/LLMRequestJsonTest.java | 61 +++++++++ tools/dependencies/known-dependencies.txt | 2 + 17 files changed, 1034 insertions(+) create mode 100644 docs/en/transform-v2/llm.md create mode 100644 docs/zh/transform-v2/llm.md create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java diff --git a/docs/en/transform-v2/llm.md b/docs/en/transform-v2/llm.md new file mode 100644 index 00000000000..d03b8226f06 --- /dev/null +++ b/docs/en/transform-v2/llm.md @@ -0,0 +1,122 @@ +# LLM + +> LLM transform plugin + +## Description + +Leverage the power of a large language model (LLM) to process data by sending it to the LLM and receiving the +generated results. Utilize the LLM's capabilities to label, clean, enrich data, perform data inference, and +more. + +## Options + +| name | type | required | default value | +|------------------|--------|----------|--------------------------------------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| prompt | string | yes | | +| model | string | yes | | +| api_key | string | yes | | +| openai.api_path | string | no | https://api.openai.com/v1/chat/completions | + +### model_provider + +The model provider to use. The available options are: +OPENAI + +### output_data_type + +The data type of the output data. The available options are: +STRING,INT,BIGINT,DOUBLE,BOOLEAN. +Default value is STRING. + +### prompt + +The prompt to send to the LLM. This parameter defines how LLM will process and return data, eg: + +The data read from source is a table like this: + +| name | age | +|---------------|-----| +| Jia Fan | 20 | +| Hailin Wang | 20 | +| Eric | 20 | +| Guangdong Liu | 20 | + +The prompt can be: + +``` +Determine whether someone is Chinese or American by their name +``` + +The result will be: + +| name | age | llm_output | +|---------------|-----|------------| +| Jia Fan | 20 | Chinese | +| Hailin Wang | 20 | Chinese | +| Eric | 20 | American | +| Guangdong Liu | 20 | Chinese | + +### model + +The model to use. Different model providers have different models. For example, the OpenAI model can be `gpt-4o-mini`. +If you use OpenAI model, please refer https://platform.openai.com/docs/models/model-endpoint-compatibility of `/v1/chat/completions` endpoint. + +### api_key + +The API key to use for the model provider. +If you use OpenAI model, please refer https://platform.openai.com/docs/api-reference/api-keys of how to get the API key. + +### openai.api_path + +The API path to use for the OpenAI model provider. In most cases, you do not need to change this configuration. If you are using an API agent's service, you may need to configure it to the agent's API address. + +### common options [string] + +Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details + +## Example + +Determine the user's country through a LLM. + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + } +} + +transform { + LLM { + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + } +} + +sink { + console { + } +} +``` + diff --git a/docs/zh/transform-v2/llm.md b/docs/zh/transform-v2/llm.md new file mode 100644 index 00000000000..acd3245b8eb --- /dev/null +++ b/docs/zh/transform-v2/llm.md @@ -0,0 +1,120 @@ +# LLM + +> LLM 转换插件 + +## 描述 + +利用大型语言模型 (LLM) 的强大功能来处理数据,方法是将数据发送到 LLM 并接收生成的结果。利用 LLM 的功能来标记、清理、丰富数据、执行数据推理等。 + +## 属性 + +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------|--------|------|--------------------------------------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| prompt | string | yes | | +| model | string | yes | | +| api_key | string | yes | | +| openai.api_path | string | no | https://api.openai.com/v1/chat/completions | + +### model_provider + +要使用的模型提供者。可用选项为: +OPENAI + +### output_data_type + +输出数据的数据类型。可用选项为: +STRING,INT,BIGINT,DOUBLE,BOOLEAN. +默认值为 STRING。 + +### prompt + +发送到 LLM 的提示。此参数定义 LLM 将如何处理和返回数据,例如: + +从源读取的数据是这样的表格: + +| name | age | +|---------------|-----| +| Jia Fan | 20 | +| Hailin Wang | 20 | +| Eric | 20 | +| Guangdong Liu | 20 | + +我们可以使用以下提示: + +``` +Determine whether someone is Chinese or American by their name +``` + +这将返回: + +| name | age | llm_output | +|---------------|-----|------------| +| Jia Fan | 20 | Chinese | +| Hailin Wang | 20 | Chinese | +| Eric | 20 | American | +| Guangdong Liu | 20 | Chinese | + +### model + +要使用的模型。不同的模型提供者有不同的模型。例如,OpenAI 模型可以是 `gpt-4o-mini`。 +如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/models/model-endpoint-compatibility 文档的`/v1/chat/completions` 端点。 + +### api_key + +用于模型提供者的 API 密钥。 +如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/api-reference/api-keys 文档的如何获取 API 密钥。 + +### openai.api_path + +用于 OpenAI 模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 + +### common options [string] + +转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 + +## 示例 + +通过 LLM 确定用户所在的国家。 + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + } +} + +transform { + LLM { + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + } +} + +sink { + console { + } +} +``` + diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java new file mode 100644 index 00000000000..6f17c5a94f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java @@ -0,0 +1,90 @@ +/* + * 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.transform; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Optional; +import java.util.stream.Stream; + +public class TestLLMIT extends TestSuiteBase implements TestResource { + private static final String TMP_DIR = "/tmp"; + private GenericContainer mockserverContainer; + private static final String IMAGE = "mockserver/mockserver:5.14.0"; + + @BeforeAll + @Override + public void startUp() { + Optional resource = + Optional.ofNullable(TestLLMIT.class.getResource("/mockserver-config.json")); + this.mockserverContainer = + new GenericContainer<>(DockerImageName.parse(IMAGE)) + .withNetwork(NETWORK) + .withNetworkAliases("mockserver") + .withExposedPorts(1080) + .withCopyFileToContainer( + MountableFile.forHostPath( + new File( + resource.orElseThrow( + () -> + new IllegalArgumentException( + "Can not get config file of mockServer")) + .getPath()) + .getAbsolutePath()), + TMP_DIR + "/mockserver-config.json") + .withEnv( + "MOCKSERVER_INITIALIZATION_JSON_PATH", + TMP_DIR + "/mockserver-config.json") + .withEnv("MOCKSERVER_LOG_LEVEL", "WARN") + .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(IMAGE))) + .waitingFor(new HttpWaitStrategy().forPath("/").forStatusCode(404)); + Startables.deepStart(Stream.of(mockserverContainer)).join(); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (mockserverContainer != null) { + mockserverContainer.stop(); + } + } + + @TestTemplate + public void testLLMWithOpenAI(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/llm_openai_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf new file mode 100644 index 00000000000..54495935893 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf @@ -0,0 +1,75 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + LLM { + source_table_name = "fake" + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + openai.api_path = "http://mockserver:1080/v1/chat/completions" + result_table_name = "llm_output" + } +} + +sink { + Assert { + source_table_name = "llm_output" + rules = + { + field_rules = [ + { + field_name = llm_output + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json new file mode 100644 index 00000000000..b4a2e53bea8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json @@ -0,0 +1,40 @@ +// https://www.mock-server.com/mock_server/getting_started.html#request_matchers + +[ + { + "httpRequest": { + "method": "POST", + "path": "/v1/chat/completions" + }, + "httpResponse": { + "body": { + "id": "chatcmpl-9s4hoBNGV0d9Mudkhvgzg64DAWPnx", + "object": "chat.completion", + "created": 1722674828, + "model": "gpt-4o-mini", + "choices": [ + { + "index": 0, + "message": { + "role": "assistant", + "content": "[\"Chinese\"]" + }, + "logprobs": null, + "finish_reason": "stop" + } + ], + "usage": { + "prompt_tokens": 107, + "completion_tokens": 3, + "total_tokens": 110 + }, + "system_fingerprint": "fp_0f03d4f0ee", + "code": 0, + "msg": "ok" + }, + "headers": { + "Content-Type": "application/json" + } + } + } +] diff --git a/seatunnel-transforms-v2/pom.xml b/seatunnel-transforms-v2/pom.xml index ae8909f463d..4cbef9a4b83 100644 --- a/seatunnel-transforms-v2/pom.xml +++ b/seatunnel-transforms-v2/pom.xml @@ -29,6 +29,11 @@ seatunnel-transforms-v2 SeaTunnel : Transforms : V2 + + 4.5.13 + 4.4.4 + + @@ -77,6 +82,16 @@ ${project.version} optional + + org.apache.httpcomponents + httpclient + ${httpclient.version} + + + org.apache.httpcomponents + httpcore + ${httpcore.version} + diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java index 0224ef4b8f7..5b97f341686 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java @@ -41,4 +41,8 @@ public RowKind getRowKind() { public Object getField(int pos) { return row.getField(pos); } + + public Object[] getFields() { + return row.getFields(); + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java new file mode 100644 index 00000000000..d19960044f1 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.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.transform.llm; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.SeaTunnelDataTypeConvertorUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; +import org.apache.seatunnel.transform.common.SingleFieldOutputTransform; +import org.apache.seatunnel.transform.llm.model.Model; +import org.apache.seatunnel.transform.llm.model.openai.OpenAIModel; + +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.util.Collections; +import java.util.List; + +public class LLMTransform extends SingleFieldOutputTransform { + private final ReadonlyConfig config; + private final SeaTunnelDataType outputDataType; + private Model model; + + public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); + this.config = config; + this.outputDataType = + SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType( + "output", config.get(LLMTransformConfig.OUTPUT_DATA_TYPE).toString()); + } + + private void tryOpen() { + if (model == null) { + open(); + } + } + + @Override + public String getPluginName() { + return "LLM"; + } + + @Override + public void open() { + ModelProvider provider = config.get(LLMTransformConfig.MODEL_PROVIDER); + if (provider.equals(ModelProvider.OPENAI)) { + model = + new OpenAIModel( + inputCatalogTable.getSeaTunnelRowType(), + outputDataType.getSqlType(), + config.get(LLMTransformConfig.PROMPT), + config.get(LLMTransformConfig.MODEL), + config.get(LLMTransformConfig.API_KEY), + config.get(LLMTransformConfig.OPENAI_API_PATH)); + } else { + throw new IllegalArgumentException("Unsupported model provider: " + provider); + } + } + + @Override + protected Object getOutputFieldValue(SeaTunnelRowAccessor inputRow) { + tryOpen(); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(inputRow.getFields()); + try { + List values = model.inference(Collections.singletonList(seaTunnelRow)); + switch (outputDataType.getSqlType()) { + case STRING: + return String.valueOf(values.get(0)); + case INT: + return Integer.parseInt(values.get(0)); + case BIGINT: + return Long.parseLong(values.get(0)); + case DOUBLE: + return Double.parseDouble(values.get(0)); + case BOOLEAN: + return Boolean.parseBoolean(values.get(0)); + default: + throw new IllegalArgumentException( + "Unsupported output data type: " + outputDataType); + } + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to inference model with row %s", seaTunnelRow), e); + } + } + + @Override + protected Column getOutputColumn() { + return PhysicalColumn.of( + "llm_output", outputDataType, (Long) null, true, null, "Output column of LLM"); + } + + @SneakyThrows + @Override + public void close() { + if (model != null) { + model.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java new file mode 100644 index 00000000000..ca3da7e6706 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.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.transform.llm; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.table.type.SqlType; + +import java.io.Serializable; + +public class LLMTransformConfig implements Serializable { + + public static final Option MODEL_PROVIDER = + Options.key("model_provider") + .enumType(ModelProvider.class) + .noDefaultValue() + .withDescription("The model provider of LLM"); + + public static final Option OUTPUT_DATA_TYPE = + Options.key("output_data_type") + .enumType(SqlType.class) + .defaultValue(SqlType.STRING) + .withDescription("The output data type of LLM"); + + public static final Option PROMPT = + Options.key("prompt") + .stringType() + .noDefaultValue() + .withDescription("The prompt of LLM"); + + public static final Option MODEL = + Options.key("model") + .stringType() + .noDefaultValue() + .withDescription( + "The model of LLM, eg: if the model provider is OpenAI, the model should be gpt-3.5-turbo/gpt-4o-mini, etc."); + + public static final Option API_KEY = + Options.key("api_key") + .stringType() + .noDefaultValue() + .withDescription("The API key of LLM"); + + public static final Option INFERENCE_BATCH_SIZE = + Options.key("inference_batch_size") + .intType() + .defaultValue(100) + .withDescription("The row batch size of each inference"); + + // OPENAI specific options + public static final Option OPENAI_API_PATH = + Options.key("openai.api_path") + .stringType() + .defaultValue("https://api.openai.com/v1/chat/completions") + .withDescription("The API path of OpenAI"); +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java new file mode 100644 index 00000000000..6fe5d53fe5b --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.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.transform.llm; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableTransform; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableTransformFactory; +import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class LLMTransformFactory implements TableTransformFactory { + @Override + public String factoryIdentifier() { + return "LLM"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + LLMTransformConfig.MODEL_PROVIDER, + LLMTransformConfig.MODEL, + LLMTransformConfig.PROMPT, + LLMTransformConfig.API_KEY) + .optional( + LLMTransformConfig.OUTPUT_DATA_TYPE, + LLMTransformConfig.INFERENCE_BATCH_SIZE) + .conditional( + LLMTransformConfig.MODEL_PROVIDER, + ModelProvider.OPENAI, + LLMTransformConfig.OPENAI_API_PATH) + .build(); + } + + @Override + public TableTransform createTransform(TableTransformFactoryContext context) { + CatalogTable catalogTable = context.getCatalogTables().get(0); + return () -> new LLMTransform(context.getOptions(), catalogTable); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java new file mode 100644 index 00000000000..a55d706c099 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.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.transform.llm; + +public enum ModelProvider { + OPENAI +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java new file mode 100644 index 00000000000..51d674c0ad8 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java @@ -0,0 +1,69 @@ +/* + * 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.transform.llm.model; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.format.json.RowToJsonConverters; + +import java.io.IOException; +import java.util.List; + +public abstract class AbstractModel implements Model { + + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final RowToJsonConverters.RowToJsonConverter rowToJsonConverters; + private final String prompt; + private final SqlType outputType; + + public AbstractModel(SeaTunnelRowType rowType, SqlType outputType, String prompt) { + this.prompt = prompt; + this.outputType = outputType; + this.rowToJsonConverters = new RowToJsonConverters().createConverter(rowType, null); + } + + private String getPromptWithLimit() { + return prompt + + "\n The following rules need to be followed: " + + "\n 1. The received data is an array, and the result is returned in the form of an array." + + "\n 2. Only the result needs to be returned, and no other information can be returned." + + "\n 3. The element type of the array is " + + outputType.toString() + + "." + + "\n Eg: [\"value1\", \"value2\"]"; + } + + @Override + public List inference(List rows) throws IOException { + ArrayNode rowsNode = OBJECT_MAPPER.createArrayNode(); + for (SeaTunnelRow row : rows) { + ObjectNode rowNode = OBJECT_MAPPER.createObjectNode(); + rowToJsonConverters.convert(OBJECT_MAPPER, rowNode, row); + rowsNode.add(rowNode); + } + return chatWithModel(getPromptWithLimit(), OBJECT_MAPPER.writeValueAsString(rowsNode)); + } + + protected abstract List chatWithModel(String promptWithLimit, String rowsJson) + throws IOException; +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java new file mode 100644 index 00000000000..77a8da63281 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm.model; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +public interface Model extends Closeable { + + List inference(List rows) throws IOException; +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java new file mode 100644 index 00000000000..9477b873202 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java @@ -0,0 +1,104 @@ +/* + * 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.transform.llm.model.openai; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.llm.model.AbstractModel; + +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; + +import com.google.common.annotations.VisibleForTesting; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; + +/** + * OpenAI model. Refer chat api + */ +@Slf4j +public class OpenAIModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String apiKey; + private final String model; + private final String apiPath; + + public OpenAIModel( + SeaTunnelRowType rowType, + SqlType outputType, + String prompt, + String model, + String apiKey, + String apiPath) { + super(rowType, outputType, prompt); + this.apiKey = apiKey; + this.apiPath = apiPath; + this.model = model; + this.client = HttpClients.createDefault(); + } + + @Override + protected List chatWithModel(String prompt, String data) throws IOException { + HttpPost post = new HttpPost(apiPath); + post.setHeader("Authorization", "Bearer " + apiKey); + post.setHeader("Content-Type", "application/json"); + ObjectNode objectNode = createJsonNodeFromData(prompt, data); + post.setEntity(new StringEntity(OBJECT_MAPPER.writeValueAsString(objectNode), "UTF-8")); + post.setConfig( + RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); + CloseableHttpResponse response = client.execute(post); + String responseStr = EntityUtils.toString(response.getEntity()); + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to chat with model, response: " + responseStr); + } + + JsonNode result = OBJECT_MAPPER.readTree(responseStr); + String resultData = result.get("choices").get(0).get("message").get("content").asText(); + return OBJECT_MAPPER.readValue(resultData, new TypeReference>() {}); + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(String prompt, String data) { + ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); + objectNode.put("model", model); + ArrayNode messages = objectNode.putArray("messages"); + messages.addObject().put("role", "system").put("content", prompt); + messages.addObject().put("role", "user").put("content", data); + return objectNode; + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java new file mode 100644 index 00000000000..39b27694805 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.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.transform; + +import org.apache.seatunnel.transform.llm.LLMTransformFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class LLMTransformFactoryTest { + + @Test + public void testOptionRule() throws Exception { + LLMTransformFactory replaceTransformFactory = new LLMTransformFactory(); + Assertions.assertNotNull(replaceTransformFactory.optionRule()); + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java new file mode 100644 index 00000000000..f32cc870559 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java @@ -0,0 +1,61 @@ +/* + * 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.transform.llm; + +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.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.llm.model.openai.OpenAIModel; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +public class LLMRequestJsonTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Test + void testOpenAIRequestJson() throws IOException { + SeaTunnelRowType rowType = + new SeaTunnelRowType( + new String[] {"id", "name"}, + new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.STRING_TYPE}); + OpenAIModel model = + new OpenAIModel( + rowType, + SqlType.STRING, + "Determine whether someone is Chinese or American by their name", + "gpt-3.5-turbo", + "sk-xxx", + "https://api.openai.com/v1/chat/completions"); + ObjectNode node = + model.createJsonNodeFromData( + "Determine whether someone is Chinese or American by their name", + "{\"id\":1, \"name\":\"John\"}"); + Assertions.assertEquals( + "{\"model\":\"gpt-3.5-turbo\",\"messages\":[{\"role\":\"system\",\"content\":\"Determine whether someone is Chinese or American by their name\"},{\"role\":\"user\",\"content\":\"{\\\"id\\\":1, \\\"name\\\":\\\"John\\\"}\"}]}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } +} diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 161134511c8..eda697369ed 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -8,6 +8,8 @@ config-1.3.3.jar disruptor-3.4.4.jar guava-27.0-jre.jar hazelcast-5.1.jar +httpclient-4.5.13.jar +httpcore-4.4.4.jar jackson-annotations-2.13.3.jar jackson-core-2.13.3.jar jackson-databind-2.13.3.jar From a4db64d7c76f07daaf8030011ee5e8552396acd2 Mon Sep 17 00:00:00 2001 From: zhangdonghao <39961809+hawk9821@users.noreply.github.com> Date: Wed, 7 Aug 2024 11:52:19 +0800 Subject: [PATCH 23/32] [Improve][E2E] Support windows for the e2e of paimon (#7329) --- .../e2e/connector/paimon/PaimonSinkCDCIT.java | 53 ++++++++++++++++--- 1 file changed, 46 insertions(+), 7 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java index c899dd0e8bf..4b1d7dd86ce 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.e2e.connector.paimon; import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.core.starter.utils.CompressionUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -25,6 +26,7 @@ import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.commons.compress.archivers.ArchiveException; import org.apache.commons.lang3.StringUtils; import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; @@ -52,6 +54,7 @@ import lombok.extern.slf4j.Slf4j; +import java.io.File; import java.io.IOException; import java.time.LocalDate; import java.util.ArrayList; @@ -68,7 +71,8 @@ "Spark and Flink engine can not auto create paimon table on worker node in local file(e.g flink tm) by savemode feature which can lead error") @Slf4j public class PaimonSinkCDCIT extends TestSuiteBase implements TestResource { - private static final String CATALOG_ROOT_DIR = "/tmp/"; + + private static String CATALOG_ROOT_DIR = "/tmp/"; private static final String NAMESPACE = "paimon"; private static final String NAMESPACE_TAR = "paimon.tar.gz"; private static final String CATALOG_DIR = CATALOG_ROOT_DIR + NAMESPACE + "/"; @@ -77,10 +81,18 @@ public class PaimonSinkCDCIT extends TestSuiteBase implements TestResource { private static final String FAKE_DATABASE1 = "FakeDatabase1"; private static final String FAKE_TABLE2 = "FakeTable1"; private static final String FAKE_DATABASE2 = "FakeDatabase2"; + private String CATALOG_ROOT_DIR_WIN = "C:/Users/"; + private String CATALOG_DIR_WIN = CATALOG_ROOT_DIR_WIN + NAMESPACE + "/"; + private boolean isWindows; @BeforeAll @Override - public void startUp() throws Exception {} + public void startUp() throws Exception { + this.isWindows = + System.getProperties().getProperty("os.name").toUpperCase().contains("WINDOWS"); + CATALOG_ROOT_DIR_WIN = CATALOG_ROOT_DIR_WIN + System.getProperty("user.name") + "/tmp/"; + CATALOG_DIR_WIN = CATALOG_ROOT_DIR_WIN + NAMESPACE + "/"; + } @AfterAll @Override @@ -498,8 +510,15 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai protected final ContainerExtendedFactory containerExtendedFactory = container -> { - FileUtils.deleteFile(CATALOG_ROOT_DIR + NAMESPACE_TAR); - FileUtils.createNewDir(CATALOG_DIR); + if (isWindows) { + FileUtils.deleteFile(CATALOG_ROOT_DIR_WIN + NAMESPACE_TAR); + FileUtils.deleteFile(CATALOG_ROOT_DIR_WIN + "paimon.tar"); + FileUtils.createNewDir(CATALOG_ROOT_DIR_WIN); + } else { + FileUtils.deleteFile(CATALOG_ROOT_DIR + NAMESPACE_TAR); + FileUtils.createNewDir(CATALOG_DIR); + } + container.execInContainer( "sh", "-c", @@ -510,8 +529,13 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai + " " + NAMESPACE); container.copyFileFromContainer( - CATALOG_ROOT_DIR + NAMESPACE_TAR, CATALOG_ROOT_DIR + NAMESPACE_TAR); - extractFiles(); + CATALOG_ROOT_DIR + NAMESPACE_TAR, + (isWindows ? CATALOG_ROOT_DIR_WIN : CATALOG_ROOT_DIR) + NAMESPACE_TAR); + if (isWindows) { + extractFilesWin(); + } else { + extractFiles(); + } }; private void extractFiles() { @@ -532,6 +556,17 @@ private void extractFiles() { } } + private void extractFilesWin() { + try { + CompressionUtils.unGzip( + new File(CATALOG_ROOT_DIR_WIN + NAMESPACE_TAR), new File(CATALOG_ROOT_DIR_WIN)); + CompressionUtils.unTar( + new File(CATALOG_ROOT_DIR_WIN + "paimon.tar"), new File(CATALOG_ROOT_DIR_WIN)); + } catch (IOException | ArchiveException e) { + throw new RuntimeException(e); + } + } + private List loadPaimonData(String dbName, String tbName) throws Exception { Table table = getTable(dbName, tbName); ReadBuilder readBuilder = table.newReadBuilder(); @@ -575,7 +610,11 @@ private Identifier getIdentifier(String dbName, String tbName) { private Catalog getCatalog() { Options options = new Options(); - options.set("warehouse", "file://" + CATALOG_DIR); + if (isWindows) { + options.set("warehouse", "file://" + CATALOG_DIR_WIN); + } else { + options.set("warehouse", "file://" + CATALOG_DIR); + } Catalog catalog = CatalogFactory.createCatalog(CatalogContext.create(options)); return catalog; } From a12786b82101ac37859deeee521b59dfd6a9cd5f Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <1058249259@qq.com> Date: Wed, 7 Aug 2024 12:27:50 +0800 Subject: [PATCH 24/32] [DOC][Oss] fix document configuration is rectified when the oss is selected as the checkpoint base (#7332) --- docs/en/seatunnel-engine/hybrid-cluster-deployment.md | 1 - docs/en/seatunnel-engine/separated-cluster-deployment.md | 1 - docs/zh/seatunnel-engine/hybrid-cluster-deployment.md | 1 - docs/zh/seatunnel-engine/separated-cluster-deployment.md | 1 - 4 files changed, 4 deletions(-) diff --git a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md index 60260f91bb0..534d5e69c5e 100644 --- a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md @@ -258,7 +258,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` Notice: When using OSS, make sure that the following jars are in the lib directory. diff --git a/docs/en/seatunnel-engine/separated-cluster-deployment.md b/docs/en/seatunnel-engine/separated-cluster-deployment.md index 6d094aa8143..168cac8d0f0 100644 --- a/docs/en/seatunnel-engine/separated-cluster-deployment.md +++ b/docs/en/seatunnel-engine/separated-cluster-deployment.md @@ -268,7 +268,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` Notice: When using OSS, make sure that the following jars are in the lib directory. diff --git a/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md b/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md index f1deba3dec1..4d101b41678 100644 --- a/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md @@ -258,7 +258,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 注意:使用OSS 时,确保 lib目录下有这几个jar. diff --git a/docs/zh/seatunnel-engine/separated-cluster-deployment.md b/docs/zh/seatunnel-engine/separated-cluster-deployment.md index 807fb8d28c7..ce328d3bd57 100644 --- a/docs/zh/seatunnel-engine/separated-cluster-deployment.md +++ b/docs/zh/seatunnel-engine/separated-cluster-deployment.md @@ -272,7 +272,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 注意:使用OSS 时,确保 lib目录下有这几个jar. From f6a1e51b890582f1cf785a40b4bb38564c5652f8 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:13 +0800 Subject: [PATCH 25/32] [E2E] Enable JdbcPostgresIdentifierIT (#7326) --- .../seatunnel/jdbc/JdbcPostgresIdentifierIT.java | 6 ------ .../connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java | 3 ++- .../seatunnel/jdbc/JdbcSqlServerCreateTableIT.java | 3 ++- .../seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java | 8 ++++---- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java index 13adec70084..a7094044aa1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java @@ -20,9 +20,7 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.AfterAll; @@ -53,10 +51,6 @@ import static org.awaitility.Awaitility.given; @Slf4j -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") public class JdbcPostgresIdentifierIT extends TestSuiteBase implements TestResource { private static final String PG_IMAGE = "postgis/postgis"; private static final String PG_DRIVER_JAR = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java index 7397362f354..30c67838975 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java @@ -59,7 +59,8 @@ @DisabledOnContainer( value = {}, type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") + disabledReason = + "Currently testcase does not depend on a specific engine, but needs to be started with the engine") public class JdbcMySqlCreateTableIT extends TestSuiteBase implements TestResource { private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; private static final String SQLSERVER_CONTAINER_HOST = "sqlserver"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java index ac8d7e31c8e..ae2e625b157 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java @@ -59,7 +59,8 @@ @DisabledOnContainer( value = {}, type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") + disabledReason = + "Currently testcase does not depend on a specific engine, but needs to be started with the engine") public class JdbcSqlServerCreateTableIT extends TestSuiteBase implements TestResource { private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java index 8fff364c3f8..b99c823de88 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java @@ -66,10 +66,6 @@ import java.util.Map; import java.util.stream.Collectors; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK}, - disabledReason = "Currently SPARK do not support cdc, temporarily disable") @Slf4j public class JdbcIrisIT extends AbstractJdbcIT { private static final String IRIS_IMAGE = "intersystems/iris-community:2023.1"; @@ -322,6 +318,10 @@ public void testCatalog() { Assertions.assertFalse(catalog.tableExists(targetTablePath)); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") @TestTemplate public void testUpsert(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/jdbc_iris_upsert.conf"); From 460e73ec36f1afac42c36320fb79a47a45bfc823 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:24 +0800 Subject: [PATCH 26/32] [E2E] Enable fakesource e2e of spark/flink (#7325) --- .../seatunnel/e2e/connector/fake/FakeWithSchemaTT.java | 6 ------ .../seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java | 6 ------ 2 files changed, 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java index b81bb7b620a..e7628d7b5da 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java @@ -18,9 +18,7 @@ package org.apache.seatunnel.e2e.connector.fake; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; @@ -28,10 +26,6 @@ import java.io.IOException; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support CatalogTable") public class FakeWithSchemaTT extends TestSuiteBase { @TestTemplate public void testFakeConnector(TestContainer container) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java index 50e58b1ae39..50b82b346e5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java @@ -18,9 +18,7 @@ package org.apache.seatunnel.e2e.connector.fake; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; @@ -28,10 +26,6 @@ import java.io.IOException; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support CatalogTable") public class FakeWithTableNamesTT extends TestSuiteBase { @TestTemplate public void testFakeConnector(TestContainer container) From 7c3cd99e007e2010fb88791cfd424d3642066a34 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:41 +0800 Subject: [PATCH 27/32] [Improve] Remove unused code (#7324) --- .../java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java | 3 --- .../java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java | 3 --- 2 files changed, 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index d4629851e79..36b25928d9b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -84,9 +84,6 @@ import java.util.stream.Stream; @Slf4j -@DisabledOnContainer( - value = {}, - disabledReason = "Override TestSuiteBase @DisabledOnContainer") public class KafkaIT extends TestSuiteBase implements TestResource { private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:7.0.9"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java index 015ab0d3e35..dc7ab433868 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java @@ -74,9 +74,6 @@ import static org.awaitility.Awaitility.await; @Slf4j -@DisabledOnContainer( - value = {}, - disabledReason = "Override TestSuiteBase @DisabledOnContainer") public class KuduIT extends TestSuiteBase implements TestResource { private static final String IMAGE = "apache/kudu:1.15.0"; From 819c6856513c9171b60d63c324e793a6eecbb9f6 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:45:54 +0800 Subject: [PATCH 28/32] [Improve][Jdbc] Merge user config primary key when create table (#7313) --- .../seatunnel/jdbc/sink/JdbcSinkFactory.java | 20 +++++++++++++++++++ .../jdbc/JdbcMysqlSaveModeHandlerIT.java | 7 +++++++ .../resources/jdbc_mysql_source_and_sink.conf | 5 ++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java index eff6bb67c67..35e9a986ab8 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -200,6 +201,25 @@ public TableSink createSink(TableSinkFactoryContext context) { .collect(Collectors.joining(","))); } } + } else { + // replace primary key to config + PrimaryKey configPk = + PrimaryKey.of( + catalogTable.getTablePath().getTableName() + "_config_pk", + config.get(PRIMARY_KEYS)); + TableSchema tableSchema = catalogTable.getTableSchema(); + catalogTable = + CatalogTable.of( + catalogTable.getTableId(), + TableSchema.builder() + .primaryKey(configPk) + .constraintKey(tableSchema.getConstraintKeys()) + .columns(tableSchema.getColumns()) + .build(), + catalogTable.getOptions(), + catalogTable.getPartitionKeys(), + catalogTable.getComment(), + catalogTable.getCatalogName()); } config = ReadonlyConfig.fromMap(new HashMap<>(map)); // always execute diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java index c8acc950105..bc1361aa267 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java @@ -47,6 +47,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -74,6 +75,7 @@ public class JdbcMysqlSaveModeHandlerIT extends AbstractJdbcIT { private static final String CREATE_SQL = "CREATE TABLE IF NOT EXISTS %s\n" + "(\n" + + " `id` bigint(20) NOT NULL,\n" + " `c_bit_1` bit(1) DEFAULT NULL,\n" + " `c_bit_8` bit(8) DEFAULT NULL,\n" + " `c_bit_16` bit(16) DEFAULT NULL,\n" @@ -164,6 +166,9 @@ void compareResult(String executeKey) { final List columns = table.getTableSchema().getColumns(); Assertions.assertEquals(columns.size(), columnsSource.size()); + Assertions.assertIterableEquals( + Collections.singletonList("id"), + table.getTableSchema().getPrimaryKey().getColumnNames()); } @Override @@ -175,6 +180,7 @@ String driverUrl() { Pair> initTestData() { String[] fieldNames = new String[] { + "id", "c_bit_1", "c_bit_8", "c_bit_16", @@ -229,6 +235,7 @@ Pair> initTestData() { SeaTunnelRow row = new SeaTunnelRow( new Object[] { + (long) i, i % 2 == 0 ? (byte) 1 : (byte) 0, new byte[] {byteArr}, new byte[] {byteArr, byteArr}, diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf index bc379f8ba8a..6305f55c46b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf @@ -40,9 +40,12 @@ sink { driver = "com.mysql.cj.jdbc.Driver" user = "root" password = "Abc!@#135_seatunnel" + generate_sink_sql = true - table = "test_laowang" database = "seatunnel" + table = "test_laowang" + primary_keys = ["id"] + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" data_save_mode="APPEND_DATA" } From 891652399e8b97fb5cf1c7e6bdee87e7ec48469e Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:46:12 +0800 Subject: [PATCH 29/32] [Hotfix][Connector] Fix kafka consumer log next startup offset (#7312) --- .../kafka/source/KafkaSourceReader.java | 10 ++- .../kafka/source/KafkaSourceSplit.java | 4 + .../e2e/connector/kafka/KafkaIT.java | 62 +++++++++++++++ ..._offset_to_console_with_commit_offset.conf | 77 +++++++++++++++++++ 4 files changed, 151 insertions(+), 2 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index 02c2a9007e1..6f4753110bd 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -218,8 +218,14 @@ public void pollNext(Collector output) throws Exception { } }); if (Boundedness.BOUNDED.equals(context.getBoundedness())) { - finishedSplits.forEach(sourceSplits::remove); - if (sourceSplits.isEmpty()) { + for (KafkaSourceSplit split : finishedSplits) { + split.setFinish(true); + if (split.getStartOffset() == -1) { + // log next running read start offset + split.setStartOffset(split.getEndOffset()); + } + } + if (sourceSplits.stream().allMatch(KafkaSourceSplit::isFinish)) { context.signalNoMoreElement(); } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java index 1c7cb17678f..8f5bc5f2d31 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java @@ -22,6 +22,9 @@ import org.apache.kafka.common.TopicPartition; +import lombok.Getter; +import lombok.Setter; + import java.util.Objects; public class KafkaSourceSplit implements SourceSplit { @@ -30,6 +33,7 @@ public class KafkaSourceSplit implements SourceSplit { private TopicPartition topicPartition; private long startOffset = -1L; private long endOffset = -1L; + @Setter @Getter private transient volatile boolean finish = false; public KafkaSourceSplit(TablePath tablePath, TopicPartition topicPartition) { this.tablePath = tablePath; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 36b25928d9b..0d9f5d5ef8a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -35,12 +35,15 @@ import org.apache.seatunnel.connectors.seatunnel.kafka.serialize.DefaultSeaTunnelRowSerializer; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.container.TestContainerId; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.format.avro.AvroDeserializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -80,6 +83,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; @@ -315,6 +319,23 @@ public void testSourceKafkaStartConfig(TestContainer container) testKafkaGroupOffsetsToConsole(container); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "flink and spark won't commit offset when batch job finished") + @TestTemplate + public void testSourceKafkaStartConfigWithCommitOffset(TestContainer container) + throws Exception { + DefaultSeaTunnelRowSerializer serializer = + DefaultSeaTunnelRowSerializer.create( + "test_topic_group_with_commit_offset", + SEATUNNEL_ROW_TYPE, + DEFAULT_FORMAT, + DEFAULT_FIELD_DELIMITER); + generateTestData(row -> serializer.serializeRow(row), 0, 100); + testKafkaGroupOffsetsToConsoleWithCommitOffset(container); + } + @TestTemplate @DisabledOnContainer(value = {TestContainerId.SPARK_2_4}) public void testFakeSourceToKafkaAvroFormat(TestContainer container) @@ -511,6 +532,40 @@ public void testKafkaGroupOffsetsToConsole(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); } + public void testKafkaGroupOffsetsToConsoleWithCommitOffset(TestContainer container) + throws IOException, InterruptedException, ExecutionException { + Container.ExecResult execResult = + container.executeJob( + "/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + String consumerGroup = "SeaTunnel-Consumer-Group"; + TopicPartition topicPartition = + new TopicPartition("test_topic_group_with_commit_offset", 0); + try (AdminClient adminClient = createKafkaAdmin()) { + ListConsumerGroupOffsetsOptions options = + new ListConsumerGroupOffsetsOptions() + .topicPartitions(Arrays.asList(topicPartition)); + Map topicOffset = + adminClient + .listConsumerGroupOffsets(consumerGroup, options) + .partitionsToOffsetAndMetadata() + .thenApply( + result -> { + Map offsets = new HashMap<>(); + result.forEach( + (tp, oam) -> { + if (oam != null) { + offsets.put(tp, oam.offset()); + } + }); + return offsets; + }) + .get(); + Assertions.assertEquals(100L, topicOffset.get(topicPartition)); + } + } + public void testKafkaTimestampToConsole(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = @@ -518,6 +573,13 @@ public void testKafkaTimestampToConsole(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); } + private AdminClient createKafkaAdmin() { + Properties props = new Properties(); + String bootstrapServers = kafkaContainer.getBootstrapServers(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + return AdminClient.create(props); + } + private void initKafkaProducer() { Properties props = new Properties(); String bootstrapServers = kafkaContainer.getBootstrapServers(); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf new file mode 100644 index 00000000000..e054c03964c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf @@ -0,0 +1,77 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Kafka { + commit_on_checkpoint = true + consumer.group = "SeaTunnel-Consumer-Group" + + bootstrap.servers = "kafkaCluster:9092" + topic = "test_topic_group_with_commit_offset" + result_table_name = "kafka_table" + # The default format is json, which is optional + format = json + start_mode = group_offsets + schema = { + fields { + id = bigint + } + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/KafkaSource +} + +transform { +} + +sink { + Assert { + source_table_name = "kafka_table" + rules = + { + field_rules = [ + { + field_name = id + field_type = bigint + field_value = [ + + { + rule_type = MIN + rule_value = 100 + }, + { + rule_type = MAX + rule_value = 149 + } + ] + } + ] + } + } +} \ No newline at end of file From 764d8b0bc8a12100bf2ef07c9b1488ba2bda70ab Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 7 Aug 2024 12:46:30 +0800 Subject: [PATCH 30/32] [Improve][API] Make sure the table name in TablePath not be null (#7252) --- .../api/sink/DefaultSaveModeHandler.java | 7 +-- .../api/table/catalog/TableIdentifier.java | 18 +++++-- .../api/table/catalog/TablePath.java | 16 ++++-- .../api/sink/TablePlaceholderTest.java | 10 ++-- .../GoogleSheetsDeserializerTest.java | 2 +- .../seatunnel/http/source/HttpSource.java | 3 +- .../source/PulsarCanalDecoratorTest.java | 2 +- .../e2e/connector/kafka/KafkaIT.java | 4 +- .../server/task/SeaTunnelSourceCollector.java | 16 ++---- .../server/task/flow/SinkFlowLifeCycle.java | 14 ++--- .../avro/AvroSerializationSchemaTest.java | 2 +- .../json/JsonRowDataSerDeSchemaTest.java | 18 +++---- .../json/canal/CanalJsonSerDeSchemaTest.java | 54 +++++++++---------- .../debezium/DebeziumJsonSerDeSchemaTest.java | 42 +++++++-------- .../maxwell/MaxWellJsonSerDeSchemaTest.java | 2 +- .../json/ogg/OggJsonSerDeSchemaTest.java | 42 +++++++-------- 16 files changed, 129 insertions(+), 123 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java index bbbe99281b2..e22dd7c99a5 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java @@ -153,19 +153,16 @@ protected void dropTable() { protected void createTable() { if (!catalog.databaseExists(tablePath.getDatabaseName())) { - TablePath databasePath = TablePath.of(tablePath.getDatabaseName(), ""); try { log.info( "Creating database {} with action {}", tablePath.getDatabaseName(), catalog.previewAction( - Catalog.ActionType.CREATE_DATABASE, - databasePath, - Optional.empty())); + Catalog.ActionType.CREATE_DATABASE, tablePath, Optional.empty())); } catch (UnsupportedOperationException ignore) { log.info("Creating database {}", tablePath.getDatabaseName()); } - catalog.createDatabase(databasePath, true); + catalog.createDatabase(tablePath, true); } try { log.info( diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java index 2d39f9b9842..101081255cc 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java @@ -17,15 +17,16 @@ package org.apache.seatunnel.api.table.catalog; +import org.apache.commons.lang3.StringUtils; + import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; +import lombok.NonNull; import java.io.Serializable; @Getter @EqualsAndHashCode -@RequiredArgsConstructor public final class TableIdentifier implements Serializable { private static final long serialVersionUID = 1L; @@ -35,7 +36,18 @@ public final class TableIdentifier implements Serializable { private final String schemaName; - private final String tableName; + @NonNull private final String tableName; + + public TableIdentifier( + String catalogName, String databaseName, String schemaName, @NonNull String tableName) { + this.catalogName = catalogName; + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + if (StringUtils.isEmpty(tableName)) { + throw new IllegalArgumentException("tableName cannot be empty"); + } + } public static TableIdentifier of(String catalogName, String databaseName, String tableName) { return new TableIdentifier(catalogName, databaseName, null, tableName); diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java index 12572621874..30edc7ac80e 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java @@ -17,9 +17,11 @@ package org.apache.seatunnel.api.table.catalog; +import org.apache.commons.lang3.StringUtils; + import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; +import lombok.NonNull; import java.io.Serializable; import java.util.ArrayList; @@ -27,12 +29,20 @@ @Getter @EqualsAndHashCode -@RequiredArgsConstructor public final class TablePath implements Serializable { private static final long serialVersionUID = 1L; private final String databaseName; private final String schemaName; - private final String tableName; + @NonNull private final String tableName; + + public TablePath(String databaseName, String schemaName, @NonNull String tableName) { + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + if (StringUtils.isEmpty(tableName)) { + throw new IllegalArgumentException("tableName cannot be empty"); + } + } public static final TablePath DEFAULT = TablePath.of("default", "default", "default"); diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java index 1a87a53f97f..16a69d5db3d 100644 --- a/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java @@ -77,7 +77,7 @@ public void testSinkOptions() { @Test public void testSinkOptionsWithNoTablePath() { ReadonlyConfig config = createConfig(); - CatalogTable table = createTestTableWithNoTablePath(); + CatalogTable table = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig = TablePlaceholder.replaceTablePlaceholder(config, table); Assertions.assertEquals("xyz_default_db_test", newConfig.get(DATABASE)); @@ -95,7 +95,7 @@ public void testSinkOptionsWithNoTablePath() { @Test public void testSinkOptionsWithExcludeKeys() { ReadonlyConfig config = createConfig(); - CatalogTable table = createTestTableWithNoTablePath(); + CatalogTable table = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig = TablePlaceholder.replaceTablePlaceholder( config, table, Arrays.asList(DATABASE.key())); @@ -116,7 +116,7 @@ public void testSinkOptionsWithExcludeKeys() { public void testSinkOptionsWithMultiTable() { ReadonlyConfig config = createConfig(); CatalogTable table1 = createTestTable(); - CatalogTable table2 = createTestTableWithNoTablePath(); + CatalogTable table2 = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig1 = TablePlaceholder.replaceTablePlaceholder(config, table1, Arrays.asList()); ReadonlyConfig newConfig2 = @@ -159,8 +159,8 @@ private static ReadonlyConfig createConfig() { return ReadonlyConfig.fromMap(configMap); } - private static CatalogTable createTestTableWithNoTablePath() { - TableIdentifier tableId = TableIdentifier.of("my-catalog", null, null, null); + private static CatalogTable createTestTableWithNoDatabaseAndSchemaName() { + TableIdentifier tableId = TableIdentifier.of("my-catalog", null, null, "default_table"); TableSchema tableSchema = TableSchema.builder() .primaryKey(PrimaryKey.of("my-pk", Arrays.asList("f1", "f2"))) diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index c55228471c2..e2e3139d887 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -42,7 +42,7 @@ public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); final DeserializationSchema deser = new JsonDeserializationSchema(catalogTables, false, false); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 754a7b93664..c41e8a9a84a 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.BasicType; @@ -146,7 +147,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { } } else { TableIdentifier tableIdentifier = - TableIdentifier.of(HttpConfig.CONNECTOR_IDENTITY, null, null); + TableIdentifier.of(HttpConfig.CONNECTOR_IDENTITY, TablePath.DEFAULT); TableSchema tableSchema = TableSchema.builder() .column( diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index 7b1ee39fd48..ee5e1513fb1 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -58,7 +58,7 @@ void decoder() throws IOException { SeaTunnelRowType seaTunnelRowType = new SeaTunnelRowType(fieldNames, dataTypes); CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", seaTunnelRowType); + CatalogTableUtil.getCatalogTable("", "", "", "test", seaTunnelRowType); CanalJsonDeserializationSchema canalJsonDeserializationSchema = CanalJsonDeserializationSchema.builder(catalogTables).build(); PulsarCanalDecorator pulsarCanalDecorator = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 0d9f5d5ef8a..6e67aa021d1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -412,7 +412,7 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) }; SeaTunnelRowType fake_source_row_type = new SeaTunnelRowType(fieldNames, fieldTypes); CatalogTable catalogTable = - CatalogTableUtil.getCatalogTable("", "", "", "", fake_source_row_type); + CatalogTableUtil.getCatalogTable("", "", "", "test", fake_source_row_type); AvroDeserializationSchema avroDeserializationSchema = new AvroDeserializationSchema(catalogTable); List kafkaSTRow = @@ -464,7 +464,7 @@ public void testKafkaAvroToAssert(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); CatalogTable catalogTable = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); AvroDeserializationSchema avroDeserializationSchema = new AvroDeserializationSchema(catalogTable); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index 62612d0617a..e1b24947893 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -102,9 +102,11 @@ public SeaTunnelSourceCollector( tablePaths.forEach( tablePath -> sourceReceivedCountPerTable.put( - getFullName(tablePath), + tablePath.getFullName(), metricsContext.counter( - SOURCE_RECEIVED_COUNT + "#" + getFullName(tablePath)))); + SOURCE_RECEIVED_COUNT + + "#" + + tablePath.getFullName()))); } sourceReceivedCount = metricsContext.counter(SOURCE_RECEIVED_COUNT); sourceReceivedQPS = metricsContext.meter(SOURCE_RECEIVED_QPS); @@ -131,7 +133,7 @@ public void collect(T row) { sourceReceivedBytesPerSeconds.markEvent(size); flowControlGate.audit((SeaTunnelRow) row); if (StringUtils.isNotEmpty(tableId)) { - String tableName = getFullName(TablePath.of(tableId)); + String tableName = TablePath.of(tableId).getFullName(); Counter sourceTableCounter = sourceReceivedCountPerTable.get(tableName); if (Objects.nonNull(sourceTableCounter)) { sourceTableCounter.inc(); @@ -232,12 +234,4 @@ public void sendRecordToNext(Record record) throws IOException { } } } - - private String getFullName(TablePath tablePath) { - if (StringUtils.isBlank(tablePath.getTableName())) { - tablePath = - TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); - } - return tablePath.getFullName(); - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index 516e1c97c41..de8257f1e94 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -138,9 +138,9 @@ public SinkFlowLifeCycle( sinkTables.forEach( tablePath -> sinkWriteCountPerTable.put( - getFullName(tablePath), + tablePath.getFullName(), metricsContext.counter( - SINK_WRITE_COUNT + "#" + getFullName(tablePath)))); + SINK_WRITE_COUNT + "#" + tablePath.getFullName()))); } } @@ -275,7 +275,7 @@ public void received(Record record) { sinkWriteBytesPerSeconds.markEvent(size); String tableId = ((SeaTunnelRow) record.getData()).getTableId(); if (StringUtils.isNotBlank(tableId)) { - String tableName = getFullName(TablePath.of(tableId)); + String tableName = TablePath.of(tableId).getFullName(); Counter sinkTableCounter = sinkWriteCountPerTable.get(tableName); if (Objects.nonNull(sinkTableCounter)) { sinkTableCounter.inc(); @@ -345,12 +345,4 @@ public void restoreState(List actionStateList) throws Except ((SupportResourceShare) this.writer).setMultiTableResourceManager(resourceManager, 0); } } - - private String getFullName(TablePath tablePath) { - if (StringUtils.isBlank(tablePath.getTableName())) { - tablePath = - TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); - } - return tablePath.getFullName(); - } } diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 1e1554be712..42b8029f16c 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -160,7 +160,7 @@ private SeaTunnelRowType buildSeaTunnelRowType() { @Test public void testSerialization() throws IOException { SeaTunnelRowType rowType = buildSeaTunnelRowType(); - CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); + CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "test", rowType); SeaTunnelRow seaTunnelRow = buildSeaTunnelRow(); AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index fb6fd9da767..beda96ff6e7 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -170,7 +170,7 @@ public void testSerDe() throws Exception { new MapType(STRING_TYPE, new MapType(STRING_TYPE, INT_TYPE)) }) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, false); @@ -230,7 +230,7 @@ public void testSerDeMultiRows() throws Exception { new SeaTunnelDataType[] {STRING_TYPE, INT_TYPE}) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, false); @@ -308,7 +308,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { new MapType(STRING_TYPE, DOUBLE_TYPE) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", rowType); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, true); @@ -327,7 +327,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { public void testDeserializationNullRow() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, true, false); @@ -339,7 +339,7 @@ public void testDeserializationNullRow() throws Exception { public void testDeserializationMissingNode() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, true, false); @@ -359,7 +359,7 @@ public void testDeserializationPassMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // pass on missing field final JsonDeserializationSchema deser = @@ -382,7 +382,7 @@ public void testDeserializationMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // fail on missing field final JsonDeserializationSchema deser = @@ -418,7 +418,7 @@ public void testDeserializationIgnoreParseError() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); SeaTunnelRow expected = new SeaTunnelRow(1); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // ignore on parse error final JsonDeserializationSchema deser = @@ -446,7 +446,7 @@ public void testDeserializationNoJson() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); String noJson = "{]"; final JsonDeserializationSchema deser = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index d35849e8bd4..efd639cd7b1 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -54,7 +54,7 @@ public class CanalJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { @@ -167,32 +167,32 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 67d499efd91..a970aea55ae 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -55,7 +55,7 @@ public class DebeziumJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test void testNullRowMessages() throws Exception { @@ -175,26 +175,26 @@ private void testSerializationDeserialization(String resourceFile, boolean schem List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index a4e06ac2b18..f82b272cf71 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -50,7 +50,7 @@ public class MaxWellJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 04fea16ecad..20df0d945ab 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -55,7 +55,7 @@ public class OggJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { @@ -172,26 +172,26 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); From 191d9e18b911a6b17e804ea26da618ed6f3d2df5 Mon Sep 17 00:00:00 2001 From: ChunFuWu <319355703@qq.com> Date: Wed, 7 Aug 2024 14:53:51 +0800 Subject: [PATCH 31/32] [FIX][SFTP] Fix username parameter error in sftp sink document (#7334) --- docs/en/connector-v2/sink/SftpFile.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/connector-v2/sink/SftpFile.md b/docs/en/connector-v2/sink/SftpFile.md index d06292bafda..7fdb542a2a6 100644 --- a/docs/en/connector-v2/sink/SftpFile.md +++ b/docs/en/connector-v2/sink/SftpFile.md @@ -36,7 +36,7 @@ By default, we use 2PC commit to ensure `exactly-once` |---------------------------------------|---------|----------|--------------------------------------------|-------------------------------------------------------------------------------------------------------------------| | host | string | yes | - | | | port | int | yes | - | | -| username | string | yes | - | | +| user | string | yes | - | | | password | string | yes | - | | | path | string | yes | - | | | tmp_path | string | yes | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a FTP dir. | @@ -72,9 +72,9 @@ The target sftp host is required The target sftp port is required -### username [string] +### user [string] -The target sftp username is required +The target sftp user is required ### password [string] @@ -229,7 +229,7 @@ For text file format with `have_partition` and `custom_filename` and `sink_colum SftpFile { host = "xxx.xxx.xxx.xxx" port = 22 - username = "username" + user = "username" password = "password" path = "/data/sftp/seatunnel/job1" tmp_path = "/data/sftp/seatunnel/tmp" From 16eeb1c12323b462474ad23103de96c0060a4537 Mon Sep 17 00:00:00 2001 From: He Wang Date: Wed, 7 Aug 2024 14:54:46 +0800 Subject: [PATCH 32/32] [Improve][Jdbc] Remove MysqlType references in JdbcDialect (#7333) --- .../jdbc/internal/dialect/JdbcDialect.java | 14 ++++---------- .../jdbc/internal/dialect/mysql/MysqlDialect.java | 7 +++---- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index e59776b6f95..f98f2cb3129 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -40,8 +40,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.mysql.cj.MysqlType; - import java.io.Serializable; import java.sql.Connection; import java.sql.PreparedStatement; @@ -532,8 +530,7 @@ default String buildAlterTableSql( "ALTER TABLE %s drop column %s", tableName, quoteIdentifier(oldColumnName)); } TypeConverter typeConverter = ConverterLoader.loadTypeConverter(dialectName()); - BasicTypeDefine typeBasicTypeDefine = - (BasicTypeDefine) typeConverter.reconvert(newColumn); + BasicTypeDefine typeBasicTypeDefine = (BasicTypeDefine) typeConverter.reconvert(newColumn); String basicSql = buildAlterTableBasicSql(alterOperation, tableName); basicSql = @@ -616,8 +613,7 @@ default String decorateWithColumnNameAndType( * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with nullable for sink table */ - default String decorateWithNullable( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithNullable(String basicSql, BasicTypeDefine typeBasicTypeDefine) { StringBuilder sql = new StringBuilder(basicSql); if (typeBasicTypeDefine.isNullable()) { sql.append("NULL "); @@ -634,8 +630,7 @@ default String decorateWithNullable( * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with default value for sink table */ - default String decorateWithDefaultValue( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithDefaultValue(String basicSql, BasicTypeDefine typeBasicTypeDefine) { Object defaultValue = typeBasicTypeDefine.getDefaultValue(); if (Objects.nonNull(defaultValue) && needsQuotesWithDefaultValue(typeBasicTypeDefine.getColumnType()) @@ -656,8 +651,7 @@ && needsQuotesWithDefaultValue(typeBasicTypeDefine.getColumnType()) * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with comment for sink table */ - default String decorateWithComment( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { String comment = typeBasicTypeDefine.getComment(); StringBuilder sql = new StringBuilder(basicSql); if (StringUtils.isNotBlank(comment)) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java index 73ef12bc47b..fd0af3d9ffd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java @@ -245,13 +245,12 @@ public void refreshTableSchemaBySchemaChangeEvent( } @Override - public String decorateWithComment( - String basicSql, BasicTypeDefine mysqlTypeBasicTypeDefine) { - MysqlType nativeType = mysqlTypeBasicTypeDefine.getNativeType(); + public String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + MysqlType nativeType = (MysqlType) typeBasicTypeDefine.getNativeType(); if (NOT_SUPPORTED_DEFAULT_VALUES.contains(nativeType)) { return basicSql; } - return JdbcDialect.super.decorateWithComment(basicSql, mysqlTypeBasicTypeDefine); + return JdbcDialect.super.decorateWithComment(basicSql, typeBasicTypeDefine); } @Override